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 2014/11/14 22:32:09 UTC

[01/58] [abbrv] [partial] incubator-calcite git commit: [CALCITE-306] Standardize code style for "import package.*; "

Repository: incubator-calcite
Updated Branches:
  refs/heads/master e3b5aa1be -> 6d79b5ebf


http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoSort.java
----------------------------------------------------------------------
diff --git a/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoSort.java b/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoSort.java
index b8f2755..c1bb26e 100644
--- a/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoSort.java
+++ b/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoSort.java
@@ -14,50 +14,48 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl.mongodb;
+package org.apache.calcite.adapter.mongodb;
 
-import org.eigenbase.rel.RelCollation;
-import org.eigenbase.rel.RelFieldCollation;
-import org.eigenbase.rel.RelNode;
-import org.eigenbase.rel.SortRel;
-import org.eigenbase.relopt.RelOptCluster;
-import org.eigenbase.relopt.RelOptCost;
-import org.eigenbase.relopt.RelOptPlanner;
-import org.eigenbase.relopt.RelTraitSet;
-import org.eigenbase.reltype.RelDataTypeField;
-import org.eigenbase.rex.RexLiteral;
-import org.eigenbase.rex.RexNode;
-import org.eigenbase.util.Util;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.util.Util;
 
 import java.util.ArrayList;
 import java.util.List;
 
 /**
-* Implementation of {@link SortRel} relational expression in MongoDB.
+* Implementation of {@link org.apache.calcite.rel.core.Sort}
+* relational expression in MongoDB.
 */
-public class MongoSortRel
-    extends SortRel
-    implements MongoRel {
-  public MongoSortRel(RelOptCluster cluster, RelTraitSet traitSet,
+public class MongoSort extends Sort implements MongoRel {
+  public MongoSort(RelOptCluster cluster, RelTraitSet traitSet,
       RelNode child, RelCollation collation, RexNode offset, RexNode fetch) {
     super(cluster, traitSet, child, collation, offset, fetch);
     assert getConvention() == MongoRel.CONVENTION;
     assert getConvention() == child.getConvention();
   }
 
-  @Override
-  public RelOptCost computeSelfCost(RelOptPlanner planner) {
+  @Override public RelOptCost computeSelfCost(RelOptPlanner planner) {
     return super.computeSelfCost(planner).multiplyBy(0.05);
   }
 
-  @Override public SortRel copy(RelTraitSet traitSet, RelNode input,
+  @Override public Sort copy(RelTraitSet traitSet, RelNode input,
       RelCollation newCollation, RexNode offset, RexNode fetch) {
-    return new MongoSortRel(getCluster(), traitSet, input, collation, offset,
+    return new MongoSort(getCluster(), traitSet, input, collation, offset,
         fetch);
   }
 
   public void implement(Implementor implementor) {
-    implementor.visitChild(0, getChild());
+    implementor.visitChild(0, getInput());
     if (!collation.getFieldCollations().isEmpty()) {
       final List<String> keys = new ArrayList<String>();
       final List<RelDataTypeField> fields = getRowType().getFieldList();
@@ -101,4 +99,4 @@ public class MongoSortRel
   }
 }
 
-// End MongoSortRel.java
+// End MongoSort.java


[55/58] [abbrv] incubator-calcite git commit: [CALCITE-460] Add ImmutableBitSet and replace uses of BitSet

Posted by jh...@apache.org.
[CALCITE-460] Add ImmutableBitSet and replace uses of BitSet


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

Branch: refs/heads/master
Commit: b9d8de38d2aa4ee6791aec42ffe2adb58942060a
Parents: a0ba73c
Author: Julian Hyde <jh...@apache.org>
Authored: Thu Nov 13 18:24:57 2014 -0800
Committer: Julian Hyde <jh...@apache.org>
Committed: Thu Nov 13 19:00:43 2014 -0800

----------------------------------------------------------------------
 .../calcite/adapter/clone/ArrayTable.java       |   9 +-
 .../apache/calcite/adapter/clone/ListTable.java |   7 +-
 .../adapter/enumerable/EnumerableAggregate.java |  14 +-
 .../adapter/enumerable/EnumerableWindow.java    |   3 +-
 .../apache/calcite/adapter/jdbc/JdbcRules.java  |  13 +-
 .../org/apache/calcite/materialize/Lattice.java |  28 +-
 .../materialize/MaterializationService.java     |  11 +-
 .../org/apache/calcite/materialize/TileKey.java |   8 +-
 .../calcite/plan/RelOptAbstractTable.java       |   4 +-
 .../org/apache/calcite/plan/RelOptLattice.java  |   4 +-
 .../org/apache/calcite/plan/RelOptTable.java    |   4 +-
 .../org/apache/calcite/plan/RelOptUtil.java     |  78 +-
 .../java/org/apache/calcite/plan/Strong.java    |  10 +-
 .../calcite/plan/SubstitutionVisitor.java       |  23 +-
 .../apache/calcite/plan/hep/HepRelVertex.java   |   4 +-
 .../apache/calcite/plan/volcano/RelSubset.java  |   4 +-
 .../apache/calcite/prepare/RelOptTableImpl.java |   4 +-
 .../org/apache/calcite/rel/AbstractRelNode.java |   7 +-
 .../java/org/apache/calcite/rel/RelInput.java   |   4 +-
 .../java/org/apache/calcite/rel/RelNode.java    |   4 +-
 .../org/apache/calcite/rel/core/Aggregate.java  |  24 +-
 .../org/apache/calcite/rel/core/Intersect.java  |   4 +-
 .../org/apache/calcite/rel/core/JoinInfo.java   |  11 +-
 .../java/org/apache/calcite/rel/core/Minus.java |   4 +-
 .../apache/calcite/rel/core/RelFactories.java   |   6 +-
 .../java/org/apache/calcite/rel/core/SetOp.java |   4 +-
 .../org/apache/calcite/rel/core/TableScan.java  |  14 +-
 .../org/apache/calcite/rel/core/Window.java     |   6 +-
 .../apache/calcite/rel/externalize/RelJson.java |   9 +-
 .../calcite/rel/externalize/RelJsonReader.java  |   7 +-
 .../calcite/rel/logical/LogicalAggregate.java   |   6 +-
 .../calcite/rel/logical/LogicalWindow.java      |  14 +-
 .../calcite/rel/metadata/BuiltInMetadata.java   |  14 +-
 .../rel/metadata/RelMdColumnUniqueness.java     |  46 +-
 .../rel/metadata/RelMdDistinctRowCount.java     |  35 +-
 .../rel/metadata/RelMdPopulationSize.java       |  39 +-
 .../calcite/rel/metadata/RelMdPredicates.java   |  54 +-
 .../calcite/rel/metadata/RelMdRowCount.java     |   8 +-
 .../calcite/rel/metadata/RelMdSelectivity.java  |   4 +-
 .../calcite/rel/metadata/RelMdUniqueKeys.java   |  71 +-
 .../apache/calcite/rel/metadata/RelMdUtil.java  | 114 +--
 .../calcite/rel/metadata/RelMetadataQuery.java  |  24 +-
 .../AggregateExpandDistinctAggregatesRule.java  |  17 +-
 .../rel/rules/AggregateFilterTransposeRule.java |  23 +-
 .../rel/rules/AggregateProjectMergeRule.java    |  11 +-
 .../AggregateProjectPullUpConstantsRule.java    |   8 +-
 .../rel/rules/AggregateStarTableRule.java       |  25 +-
 .../rel/rules/FilterAggregateTransposeRule.java |   9 +-
 .../calcite/rel/rules/JoinAssociateRule.java    |   8 +-
 .../rel/rules/JoinPushThroughJoinRule.java      |  12 +-
 .../calcite/rel/rules/JoinToMultiJoinRule.java  |   6 +-
 .../apache/calcite/rel/rules/LoptMultiJoin.java |  88 +-
 .../calcite/rel/rules/LoptOptimizeJoinRule.java |  74 +-
 .../rel/rules/LoptSemiJoinOptimizer.java        |  23 +-
 .../org/apache/calcite/rel/rules/MultiJoin.java |   8 +-
 .../rel/rules/MultiJoinOptimizeBushyRule.java   |  44 +-
 .../apache/calcite/rel/rules/PushProjector.java |  25 +-
 .../apache/calcite/rel/rules/SemiJoinRule.java  |  17 +-
 .../org/apache/calcite/schema/Statistic.java    |   4 +-
 .../org/apache/calcite/schema/Statistics.java   |  14 +-
 .../apache/calcite/sql2rel/RelDecorrelator.java |  13 +-
 .../apache/calcite/sql2rel/RelFieldTrimmer.java | 121 ++-
 .../calcite/sql2rel/SqlToRelConverter.java      |  15 +-
 .../java/org/apache/calcite/util/BitSets.java   |  22 +
 .../org/apache/calcite/util/BuiltInMethod.java  |  10 +-
 .../apache/calcite/util/ImmutableBitSet.java    | 882 +++++++++++++++++++
 .../apache/calcite/util/mapping/Mappings.java   |  23 +
 .../org/apache/calcite/plan/RelWriterTest.java  |   4 +-
 .../org/apache/calcite/test/CalciteSuite.java   |   2 +
 .../apache/calcite/test/MockCatalogReader.java  |   4 +-
 .../apache/calcite/test/RelMetadataTest.java    |   4 +-
 .../org/apache/calcite/test/RexProgramTest.java |  15 +-
 .../apache/calcite/test/SqlToRelTestBase.java   |   6 +-
 .../calcite/util/ImmutableBitSetTest.java       | 383 ++++++++
 .../calcite/adapter/mongodb/MongoAggregate.java |  13 +-
 .../calcite/adapter/tpcds/TpcdsSchema.java      |   4 +-
 76 files changed, 1975 insertions(+), 723 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/core/src/main/java/org/apache/calcite/adapter/clone/ArrayTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/clone/ArrayTable.java b/core/src/main/java/org/apache/calcite/adapter/clone/ArrayTable.java
index 6bfe42f..7c7ccf3 100644
--- a/core/src/main/java/org/apache/calcite/adapter/clone/ArrayTable.java
+++ b/core/src/main/java/org/apache/calcite/adapter/clone/ArrayTable.java
@@ -29,18 +29,17 @@ import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.schema.Statistic;
 import org.apache.calcite.schema.Statistics;
 import org.apache.calcite.schema.impl.AbstractTableQueryable;
-import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.Pair;
 
 import com.google.common.base.Supplier;
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
 
 import java.lang.reflect.Array;
 import java.lang.reflect.Type;
 import java.util.AbstractList;
-import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.BitSet;
 import java.util.Collections;
 import java.util.List;
 
@@ -67,11 +66,11 @@ class ArrayTable extends AbstractQueryableTable {
   }
 
   public Statistic getStatistic() {
-    final List<BitSet> keys = new ArrayList<BitSet>();
+    final List<ImmutableBitSet> keys = Lists.newArrayList();
     final Content content = supplier.get();
     for (Ord<Column> ord : Ord.zip(content.columns)) {
       if (ord.e.cardinality == content.size) {
-        keys.add(BitSets.of(ord.i));
+        keys.add(ImmutableBitSet.of(ord.i));
       }
     }
     return Statistics.of(content.size, keys);

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/core/src/main/java/org/apache/calcite/adapter/clone/ListTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/clone/ListTable.java b/core/src/main/java/org/apache/calcite/adapter/clone/ListTable.java
index c07ea0a..cb446ff 100644
--- a/core/src/main/java/org/apache/calcite/adapter/clone/ListTable.java
+++ b/core/src/main/java/org/apache/calcite/adapter/clone/ListTable.java
@@ -29,10 +29,11 @@ import org.apache.calcite.rel.type.RelProtoDataType;
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.schema.Statistic;
 import org.apache.calcite.schema.Statistics;
+import org.apache.calcite.util.ImmutableBitSet;
+
+import com.google.common.collect.ImmutableList;
 
 import java.lang.reflect.Type;
-import java.util.BitSet;
-import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
 
@@ -63,7 +64,7 @@ class ListTable extends AbstractQueryableTable {
   }
 
   public Statistic getStatistic() {
-    return Statistics.of(list.size(), Collections.<BitSet>emptyList());
+    return Statistics.of(list.size(), ImmutableList.<ImmutableBitSet>of());
   }
 
   public <T> Queryable<T> asQueryable(final QueryProvider queryProvider,

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableAggregate.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableAggregate.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableAggregate.java
index 7f19cdf..91014bf 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableAggregate.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableAggregate.java
@@ -39,8 +39,8 @@ import org.apache.calcite.rex.RexInputRef;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
-import org.apache.calcite.util.BitSets;
 import org.apache.calcite.util.BuiltInMethod;
+import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Util;
 
@@ -48,7 +48,6 @@ import com.google.common.collect.ImmutableList;
 
 import java.lang.reflect.Type;
 import java.util.ArrayList;
-import java.util.BitSet;
 import java.util.Collections;
 import java.util.List;
 
@@ -67,7 +66,7 @@ public class EnumerableAggregate extends Aggregate
       RelOptCluster cluster,
       RelTraitSet traitSet,
       RelNode child,
-      BitSet groupSet,
+      ImmutableBitSet groupSet,
       List<AggregateCall> aggCalls)
       throws InvalidRelException {
     super(cluster, traitSet, child, groupSet, aggCalls);
@@ -88,7 +87,7 @@ public class EnumerableAggregate extends Aggregate
   }
 
   @Override public EnumerableAggregate copy(RelTraitSet traitSet,
-      RelNode input, BitSet groupSet, List<AggregateCall> aggCalls) {
+      RelNode input, ImmutableBitSet groupSet, List<AggregateCall> aggCalls) {
     try {
       return new EnumerableAggregate(getCluster(), traitSet, input,
           groupSet, aggCalls);
@@ -179,15 +178,14 @@ public class EnumerableAggregate extends Aggregate
         Expressions.parameter(inputPhysType.getJavaRowType(), "a0");
 
     final PhysType keyPhysType =
-        inputPhysType.project(
-            BitSets.toList(groupSet), JavaRowFormat.LIST);
+        inputPhysType.project(groupSet.toList(), JavaRowFormat.LIST);
     final int keyArity = groupSet.cardinality();
     final Expression keySelector =
         builder.append(
             "keySelector",
             inputPhysType.generateSelector(
                 parameter,
-                BitSets.toList(groupSet),
+                groupSet.toList(),
                 keyPhysType.getFormat()));
 
     final List<AggImpState> aggs =
@@ -385,7 +383,7 @@ public class EnumerableAggregate extends Aggregate
                       resultSelector))));
     } else if (aggCalls.isEmpty()
         && groupSet.equals(
-            BitSets.range(child.getRowType().getFieldCount()))) {
+            ImmutableBitSet.range(child.getRowType().getFieldCount()))) {
       builder.add(
           Expressions.return_(
               null,

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableWindow.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableWindow.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableWindow.java
index de0ddd1..fd0304b 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableWindow.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableWindow.java
@@ -46,7 +46,6 @@ import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexWindowBound;
 import org.apache.calcite.runtime.SortedMultiMap;
-import org.apache.calcite.util.BitSets;
 import org.apache.calcite.util.BuiltInMethod;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Util;
@@ -697,7 +696,7 @@ public class EnumerableWindow extends Window implements EnumerableRel {
             0, "key",
             inputPhysType.selector(
                 v_,
-                BitSets.toList(group.keys),
+                group.keys.toList(),
                 JavaRowFormat.CUSTOM));
     builder2.add(declare);
     final ParameterExpression key_ = declare.parameter;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcRules.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcRules.java b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcRules.java
index f793a43..a085429 100644
--- a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcRules.java
+++ b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcRules.java
@@ -88,7 +88,7 @@ import org.apache.calcite.sql.type.InferTypes;
 import org.apache.calcite.sql.type.OperandTypes;
 import org.apache.calcite.sql.type.ReturnTypes;
 import org.apache.calcite.sql.validate.SqlValidatorUtil;
-import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.ImmutableIntList;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.trace.CalciteTrace;
@@ -96,7 +96,6 @@ import org.apache.calcite.util.trace.CalciteTrace;
 import com.google.common.collect.ImmutableList;
 
 import java.util.ArrayList;
-import java.util.BitSet;
 import java.util.Collections;
 import java.util.List;
 import java.util.Set;
@@ -271,8 +270,8 @@ public class JdbcRules {
     }
 
     @Override public double getRows() {
-      final boolean leftKey = left.isKey(BitSets.of(leftKeys));
-      final boolean rightKey = right.isKey(BitSets.of(rightKeys));
+      final boolean leftKey = left.isKey(ImmutableBitSet.of(leftKeys));
+      final boolean rightKey = right.isKey(ImmutableBitSet.of(rightKeys));
       final double leftRowCount = left.getRows();
       final double rightRowCount = right.getRows();
       if (leftKey && rightKey) {
@@ -566,7 +565,7 @@ public class JdbcRules {
         RelOptCluster cluster,
         RelTraitSet traitSet,
         RelNode child,
-        BitSet groupSet,
+        ImmutableBitSet groupSet,
         List<AggregateCall> aggCalls)
         throws InvalidRelException {
       super(cluster, traitSet, child, groupSet, aggCalls);
@@ -574,7 +573,7 @@ public class JdbcRules {
     }
 
     @Override public JdbcAggregate copy(RelTraitSet traitSet, RelNode input,
-        BitSet groupSet, List<AggregateCall> aggCalls) {
+        ImmutableBitSet groupSet, List<AggregateCall> aggCalls) {
       try {
         return new JdbcAggregate(getCluster(), traitSet, input, groupSet,
             aggCalls);
@@ -592,7 +591,7 @@ public class JdbcRules {
           x.builder(this, JdbcImplementor.Clause.GROUP_BY);
       List<SqlNode> groupByList = Expressions.list();
       final List<SqlNode> selectList = new ArrayList<SqlNode>();
-      for (int group : BitSets.toIter(groupSet)) {
+      for (int group : groupSet) {
         final SqlNode field = builder.context.field(group);
         addSelect(selectList, field, getRowType());
         groupByList.add(field);

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/core/src/main/java/org/apache/calcite/materialize/Lattice.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/materialize/Lattice.java b/core/src/main/java/org/apache/calcite/materialize/Lattice.java
index b4a75c3..f1d926a 100644
--- a/core/src/main/java/org/apache/calcite/materialize/Lattice.java
+++ b/core/src/main/java/org/apache/calcite/materialize/Lattice.java
@@ -44,6 +44,7 @@ import org.apache.calcite.sql.SqlUtil;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.validate.SqlValidatorUtil;
 import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.Util;
 import org.apache.calcite.util.graph.DefaultDirectedGraph;
 import org.apache.calcite.util.graph.DefaultEdge;
@@ -61,7 +62,6 @@ import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 
 import java.math.BigInteger;
-import java.util.BitSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -228,18 +228,22 @@ public class Lattice {
     throw new AssertionError("input not found");
   }
 
-  public String sql(BitSet groupSet, List<Measure> aggCallList) {
-    final BitSet columnSet = (BitSet) groupSet.clone();
+  public String sql(ImmutableBitSet groupSet, List<Measure> aggCallList) {
+    final ImmutableBitSet.Builder columnSetBuilder =
+        ImmutableBitSet.builder(groupSet);
     for (Measure call : aggCallList) {
       for (Column arg : call.args) {
-        columnSet.set(arg.ordinal);
+        columnSetBuilder.set(arg.ordinal);
       }
     }
+    final ImmutableBitSet columnSet = columnSetBuilder.build();
+
     // Figure out which nodes are needed. Use a node if its columns are used
     // or if has a child whose columns are used.
     List<Node> usedNodes = Lists.newArrayList();
     for (Node node : nodes) {
-      if (BitSets.range(node.startCol, node.endCol).intersects(columnSet)) {
+      if (ImmutableBitSet.range(node.startCol, node.endCol)
+          .intersects(columnSet)) {
         use(usedNodes, node);
       }
     }
@@ -522,12 +526,12 @@ public class Lattice {
     }
 
     /** Returns the set of distinct argument ordinals. */
-    public BitSet argBitSet() {
-      final BitSet bitSet = new BitSet();
+    public ImmutableBitSet argBitSet() {
+      final ImmutableBitSet.Builder bitSet = ImmutableBitSet.builder();
       for (Column arg : args) {
         bitSet.set(arg.ordinal);
       }
-      return bitSet;
+      return bitSet.build();
     }
 
     /** Returns a list of argument ordinals. */
@@ -819,7 +823,7 @@ public class Lattice {
   public static class Tile {
     public final ImmutableList<Measure> measures;
     public final ImmutableList<Column> dimensions;
-    public final BitSet bitSet = new BitSet();
+    public final ImmutableBitSet bitSet;
 
     public Tile(ImmutableList<Measure> measures,
         ImmutableList<Column> dimensions) {
@@ -827,16 +831,18 @@ public class Lattice {
       this.dimensions = dimensions;
       assert Util.isStrictlySorted(dimensions);
       assert Util.isStrictlySorted(measures);
+      final ImmutableBitSet.Builder bitSetBuilder = ImmutableBitSet.builder();
       for (Column dimension : dimensions) {
-        bitSet.set(dimension.ordinal);
+        bitSetBuilder.set(dimension.ordinal);
       }
+      bitSet = bitSetBuilder.build();
     }
 
     public static TileBuilder builder() {
       return new TileBuilder();
     }
 
-    public BitSet bitSet() {
+    public ImmutableBitSet bitSet() {
       return bitSet;
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/core/src/main/java/org/apache/calcite/materialize/MaterializationService.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/materialize/MaterializationService.java b/core/src/main/java/org/apache/calcite/materialize/MaterializationService.java
index a4e289c..c142454 100644
--- a/core/src/main/java/org/apache/calcite/materialize/MaterializationService.java
+++ b/core/src/main/java/org/apache/calcite/materialize/MaterializationService.java
@@ -37,7 +37,7 @@ import org.apache.calcite.rel.type.RelDataTypeImpl;
 import org.apache.calcite.runtime.Hook;
 import org.apache.calcite.schema.Schemas;
 import org.apache.calcite.schema.Table;
-import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Util;
 
@@ -48,7 +48,6 @@ import com.google.common.collect.Sets;
 
 import java.lang.reflect.Type;
 import java.util.ArrayList;
-import java.util.BitSet;
 import java.util.Comparator;
 import java.util.Iterator;
 import java.util.LinkedHashSet;
@@ -199,8 +198,8 @@ public class MaterializationService {
    * materialization would try to create itself to populate itself!
    */
   public Pair<CalciteSchema.TableEntry, TileKey> defineTile(Lattice lattice,
-      BitSet groupSet, List<Lattice.Measure> measureList, CalciteSchema schema,
-      boolean create, boolean exact) {
+      ImmutableBitSet groupSet, List<Lattice.Measure> measureList,
+      CalciteSchema schema, boolean create, boolean exact) {
     MaterializationKey materializationKey;
     final TileKey tileKey =
         new TileKey(lattice, groupSet, ImmutableList.copyOf(measureList));
@@ -251,7 +250,7 @@ public class MaterializationService {
           : actor.keyByTile.entrySet()) {
         final TileKey tileKey2 = entry.getKey();
         if (tileKey2.lattice == lattice
-            && BitSets.contains(tileKey2.dimensions, groupSet)
+            && tileKey2.dimensions.contains(groupSet)
             && !tileKey2.dimensions.equals(groupSet)
             && allSatisfiable(measureList, tileKey2)) {
           materializationKey = entry.getValue();
@@ -321,7 +320,7 @@ public class MaterializationService {
     // less obviously, if it is composed of grouping columns.
     for (Lattice.Measure measure : measureList) {
       if (!(tileKey.measures.contains(measure)
-          || BitSets.contains(tileKey.dimensions, measure.argBitSet()))) {
+          || tileKey.dimensions.contains(measure.argBitSet()))) {
         return false;
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/core/src/main/java/org/apache/calcite/materialize/TileKey.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/materialize/TileKey.java b/core/src/main/java/org/apache/calcite/materialize/TileKey.java
index 642edb6..1db02a9 100644
--- a/core/src/main/java/org/apache/calcite/materialize/TileKey.java
+++ b/core/src/main/java/org/apache/calcite/materialize/TileKey.java
@@ -16,11 +16,11 @@
  */
 package org.apache.calcite.materialize;
 
+import org.apache.calcite.util.ImmutableBitSet;
+
 import com.google.common.base.Objects;
 import com.google.common.collect.ImmutableList;
 
-import java.util.BitSet;
-
 /** Definition of a particular combination of dimensions and measures of a
  * lattice that is the basis of a materialization.
  *
@@ -29,11 +29,11 @@ import java.util.BitSet;
  * immutable and tiles are not added after their creation. */
 public class TileKey {
   public final Lattice lattice;
-  public final BitSet dimensions;
+  public final ImmutableBitSet dimensions;
   public final ImmutableList<Lattice.Measure> measures;
 
   /** Creates a TileKey. */
-  public TileKey(Lattice lattice, BitSet dimensions,
+  public TileKey(Lattice lattice, ImmutableBitSet dimensions,
       ImmutableList<Lattice.Measure> measures) {
     this.lattice = lattice;
     this.dimensions = dimensions;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/core/src/main/java/org/apache/calcite/plan/RelOptAbstractTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/RelOptAbstractTable.java b/core/src/main/java/org/apache/calcite/plan/RelOptAbstractTable.java
index 5496477..2c3099d 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptAbstractTable.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptAbstractTable.java
@@ -21,10 +21,10 @@ import org.apache.calcite.rel.RelCollation;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.logical.LogicalTableScan;
 import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.util.ImmutableBitSet;
 
 import com.google.common.collect.ImmutableList;
 
-import java.util.BitSet;
 import java.util.Collections;
 import java.util.List;
 
@@ -83,7 +83,7 @@ public abstract class RelOptAbstractTable implements RelOptTable {
   }
 
   // Override to define keys
-  public boolean isKey(BitSet columns) {
+  public boolean isKey(ImmutableBitSet columns) {
     return false;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/core/src/main/java/org/apache/calcite/plan/RelOptLattice.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/RelOptLattice.java b/core/src/main/java/org/apache/calcite/plan/RelOptLattice.java
index 9131ba3..a32f55d 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptLattice.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptLattice.java
@@ -22,9 +22,9 @@ import org.apache.calcite.materialize.Lattice;
 import org.apache.calcite.materialize.MaterializationService;
 import org.apache.calcite.materialize.TileKey;
 import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.Pair;
 
-import java.util.BitSet;
 import java.util.List;
 
 /**
@@ -70,7 +70,7 @@ public class RelOptLattice {
    * @return Materialized table
    */
   public Pair<CalciteSchema.TableEntry, TileKey> getAggregate(
-      RelOptPlanner planner, BitSet groupSet,
+      RelOptPlanner planner, ImmutableBitSet groupSet,
       List<Lattice.Measure> measureList) {
     final CalciteConnectionConfig config =
         planner.getContext().unwrap(CalciteConnectionConfig.class);

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/core/src/main/java/org/apache/calcite/plan/RelOptTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/RelOptTable.java b/core/src/main/java/org/apache/calcite/plan/RelOptTable.java
index 571fbd1..c628f71 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptTable.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptTable.java
@@ -20,8 +20,8 @@ import org.apache.calcite.linq4j.tree.Expression;
 import org.apache.calcite.rel.RelCollation;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.util.ImmutableBitSet;
 
-import java.util.BitSet;
 import java.util.List;
 
 /**
@@ -82,7 +82,7 @@ public interface RelOptTable {
    * @param columns Ordinals of key columns
    * @return Whether the given columns are a key or a superset of a key
    */
-  boolean isKey(BitSet columns);
+  boolean isKey(ImmutableBitSet columns);
 
   /**
    * Finds an interface implemented by this table.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/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 e188d35..5949dbf 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
@@ -67,9 +67,9 @@ import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.type.MultisetSqlType;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.validate.SqlValidatorUtil;
-import org.apache.calcite.util.BitSets;
 import org.apache.calcite.util.Bug;
 import org.apache.calcite.util.Holder;
+import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Permutation;
 import org.apache.calcite.util.Util;
@@ -360,7 +360,7 @@ public abstract class RelOptUtil {
           new LogicalAggregate(
               ret.getCluster(),
               ret,
-              BitSets.of(),
+              ImmutableBitSet.of(),
               ImmutableList.of(aggCall));
     }
 
@@ -401,7 +401,7 @@ public abstract class RelOptUtil {
       final int keyCount = ret.getRowType().getFieldCount();
       if (!needsOuterJoin) {
         return Pair.<RelNode, Boolean>of(
-            new LogicalAggregate(cluster, ret, BitSets.range(keyCount),
+            new LogicalAggregate(cluster, ret, ImmutableBitSet.range(keyCount),
                 ImmutableList.<AggregateCall>of()),
             false);
       }
@@ -438,7 +438,7 @@ public abstract class RelOptUtil {
       ret = new LogicalAggregate(
           cluster,
           ret,
-          BitSets.range(projectedKeyCount),
+          ImmutableBitSet.range(projectedKeyCount),
           ImmutableList.of(aggCall));
 
       switch (logic) {
@@ -683,7 +683,7 @@ public abstract class RelOptUtil {
     return new LogicalAggregate(
         rel.getCluster(),
         rel,
-        BitSets.of(),
+        ImmutableBitSet.of(),
         aggCalls);
   }
 
@@ -698,7 +698,7 @@ public abstract class RelOptUtil {
     return new LogicalAggregate(
         rel.getCluster(),
         rel,
-        BitSets.range(rel.getRowType().getFieldCount()),
+        ImmutableBitSet.range(rel.getRowType().getFieldCount()),
         ImmutableList.<AggregateCall>of());
   }
 
@@ -967,8 +967,8 @@ public abstract class RelOptUtil {
         RexNode op0 = operands.get(0);
         RexNode op1 = operands.get(1);
 
-        final BitSet projRefs0 = RelOptUtil.InputFinder.bits(op0);
-        final BitSet projRefs1 = RelOptUtil.InputFinder.bits(op1);
+        final ImmutableBitSet projRefs0 = InputFinder.bits(op0);
+        final ImmutableBitSet projRefs1 = InputFinder.bits(op1);
 
         if ((projRefs0.nextSetBit(firstRightField) < 0)
             && (projRefs1.nextSetBit(firstLeftField)
@@ -1041,7 +1041,7 @@ public abstract class RelOptUtil {
         //     f(LHS) > 0 ===> ( f(LHS) > 0 ) = TRUE,
         // and make the RHS produce TRUE, but only if we're strictly
         // looking for equi-joins
-        final BitSet projRefs = RelOptUtil.InputFinder.bits(condition);
+        final ImmutableBitSet projRefs = InputFinder.bits(condition);
         leftKey = null;
         rightKey = null;
 
@@ -2008,8 +2008,10 @@ public abstract class RelOptUtil {
     assert nTotalFields == nSysFields + nFieldsLeft + nFieldsRight;
 
     // set the reference bitmaps for the left and right children
-    BitSet leftBitmap = BitSets.range(nSysFields, nSysFields + nFieldsLeft);
-    BitSet rightBitmap = BitSets.range(nSysFields + nFieldsLeft, nTotalFields);
+    ImmutableBitSet leftBitmap =
+        ImmutableBitSet.range(nSysFields, nSysFields + nFieldsLeft);
+    ImmutableBitSet rightBitmap =
+        ImmutableBitSet.range(nSysFields + nFieldsLeft, nTotalFields);
 
     for (RexNode filter : aboveFilters) {
       if (joinType.generatesNullsOnLeft()
@@ -2117,14 +2119,15 @@ public abstract class RelOptUtil {
         : nSysFields + nFieldsLeft + nFieldsRight);
 
     // set the reference bitmaps for the left and right children
-    BitSet leftBitmap =
-        BitSets.range(nSysFields, nSysFields + nFieldsLeft);
-    BitSet rightBitmap =
-        BitSets.range(nSysFields + nFieldsLeft, nTotalFields);
+    ImmutableBitSet leftBitmap =
+        ImmutableBitSet.range(nSysFields, nSysFields + nFieldsLeft);
+    ImmutableBitSet rightBitmap =
+        ImmutableBitSet.range(nSysFields + nFieldsLeft, nTotalFields);
 
     final List<RexNode> filtersToRemove = Lists.newArrayList();
     for (RexNode filter : filters) {
       final InputFinder inputFinder = InputFinder.analyze(filter);
+      final ImmutableBitSet inputBits = inputFinder.inputBitSet.build();
 
       // REVIEW - are there any expressions that need special handling
       // and therefore cannot be pushed?
@@ -2132,7 +2135,7 @@ public abstract class RelOptUtil {
       // filters can be pushed to the left child if the left child
       // does not generate NULLs and the only columns referenced in
       // the filter originate from the left child
-      if (pushLeft && BitSets.contains(leftBitmap, inputFinder.inputBitSet)) {
+      if (pushLeft && leftBitmap.contains(inputBits)) {
         // ignore filters that always evaluate to true
         if (!filter.isAlwaysTrue()) {
           // adjust the field references in the filter to reflect
@@ -2156,8 +2159,7 @@ public abstract class RelOptUtil {
         // filters can be pushed to the right child if the right child
         // does not generate NULLs and the only columns referenced in
         // the filter originate from the right child
-      } else if (pushRight
-          && BitSets.contains(rightBitmap, inputFinder.inputBitSet)) {
+      } else if (pushRight && rightBitmap.contains(inputBits)) {
         if (!filter.isAlwaysTrue()) {
           // adjust the field references in the filter to reflect
           // that fields in the right now shift over to the left;
@@ -2234,15 +2236,15 @@ public abstract class RelOptUtil {
    *                    the child input
    */
   public static void splitFilters(
-      BitSet childBitmap,
+      ImmutableBitSet childBitmap,
       RexNode predicate,
       List<RexNode> pushable,
       List<RexNode> notPushable) {
     // for each filter, if the filter only references the child inputs,
     // then it can be pushed
     for (RexNode filter : conjunctions(predicate)) {
-      BitSet filterRefs = RelOptUtil.InputFinder.bits(filter);
-      if (BitSets.contains(childBitmap, filterRefs)) {
+      ImmutableBitSet filterRefs = InputFinder.bits(filter);
+      if (childBitmap.contains(filterRefs)) {
         pushable.add(filter);
       } else {
         notPushable.add(filter);
@@ -2447,12 +2449,12 @@ public abstract class RelOptUtil {
     // the post-join filter.  Since the filter effectively sits in
     // between the LogicalProject and the MultiJoin, the projection needs
     // to include those filter references.
-    BitSet inputRefs = InputFinder.bits(
-        project.getProjects(), multiJoin.getPostJoinFilter());
+    ImmutableBitSet inputRefs =
+        InputFinder.bits(project.getProjects(), multiJoin.getPostJoinFilter());
 
     // create new copies of the bitmaps
     List<RelNode> multiJoinInputs = multiJoin.getInputs();
-    List<BitSet> newProjFields = new ArrayList<BitSet>();
+    List<BitSet> newProjFields = Lists.newArrayList();
     for (RelNode multiJoinInput : multiJoinInputs) {
       newProjFields.add(
           new BitSet(multiJoinInput.getRowType().getFieldCount()));
@@ -2462,7 +2464,7 @@ public abstract class RelOptUtil {
     int currInput = -1;
     int startField = 0;
     int nFields = 0;
-    for (int bit : BitSets.toIter(inputRefs)) {
+    for (int bit : inputRefs) {
       while (bit >= (startField + nFields)) {
         startField += nFields;
         currInput++;
@@ -2483,7 +2485,7 @@ public abstract class RelOptUtil {
         multiJoin.isFullOuterJoin(),
         multiJoin.getOuterJoinConditions(),
         multiJoin.getJoinTypes(),
-        newProjFields,
+        Lists.transform(newProjFields, ImmutableBitSet.FROM_BIT_SET),
         multiJoin.getJoinFieldRefCountsMap(),
         multiJoin.getPostJoinFilter());
   }
@@ -3071,22 +3073,22 @@ public abstract class RelOptUtil {
    * Visitor which builds a bitmap of the inputs used by an expression.
    */
   public static class InputFinder extends RexVisitorImpl<Void> {
-    final BitSet inputBitSet;
+    public final ImmutableBitSet.Builder inputBitSet;
     private final Set<RelDataTypeField> extraFields;
 
-    public InputFinder(BitSet inputBitSet) {
-      this(inputBitSet, null);
+    public InputFinder() {
+      this(null);
     }
 
-    public InputFinder(BitSet inputBitSet, Set<RelDataTypeField> extraFields) {
+    public InputFinder(Set<RelDataTypeField> extraFields) {
       super(true);
-      this.inputBitSet = inputBitSet;
+      this.inputBitSet = ImmutableBitSet.builder();
       this.extraFields = extraFields;
     }
 
     /** Returns an input finder that has analyzed a given expression. */
     public static InputFinder analyze(RexNode node) {
-      final InputFinder inputFinder = new InputFinder(new BitSet());
+      final InputFinder inputFinder = new InputFinder();
       node.accept(inputFinder);
       return inputFinder;
     }
@@ -3094,18 +3096,18 @@ public abstract class RelOptUtil {
     /**
      * Returns a bit set describing the inputs used by an expression.
      */
-    public static BitSet bits(RexNode node) {
-      return analyze(node).inputBitSet;
+    public static ImmutableBitSet bits(RexNode node) {
+      return analyze(node).inputBitSet.build();
     }
 
     /**
      * Returns a bit set describing the inputs used by a collection of
      * project expressions and an optional condition.
      */
-    public static BitSet bits(List<RexNode> exprs, RexNode expr) {
-      final BitSet inputBitSet = new BitSet();
-      RexProgram.apply(new InputFinder(inputBitSet), exprs, expr);
-      return inputBitSet;
+    public static ImmutableBitSet bits(List<RexNode> exprs, RexNode expr) {
+      final InputFinder inputFinder = new InputFinder();
+      RexProgram.apply(inputFinder, exprs, expr);
+      return inputFinder.inputBitSet.build();
     }
 
     public Void visitInputRef(RexInputRef inputRef) {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/core/src/main/java/org/apache/calcite/plan/Strong.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/Strong.java b/core/src/main/java/org/apache/calcite/plan/Strong.java
index e279b14..5957bc7 100644
--- a/core/src/main/java/org/apache/calcite/plan/Strong.java
+++ b/core/src/main/java/org/apache/calcite/plan/Strong.java
@@ -20,8 +20,8 @@ import org.apache.calcite.rex.RexCall;
 import org.apache.calcite.rex.RexInputRef;
 import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.util.ImmutableBitSet;
 
-import java.util.BitSet;
 import java.util.List;
 
 /** Utilities for strong predicates.
@@ -42,19 +42,19 @@ import java.util.List;
  * </ul>
  */
 public class Strong {
-  private final BitSet nullColumns;
+  private final ImmutableBitSet nullColumns;
 
-  private Strong(BitSet nullColumns) {
+  private Strong(ImmutableBitSet nullColumns) {
     this.nullColumns = nullColumns;
   }
 
-  public static Strong of(BitSet nullColumns) {
+  public static Strong of(ImmutableBitSet nullColumns) {
     return new Strong(nullColumns);
   }
 
   /** Returns whether the analyzed expression will return null if a given set
    * of input columns are null. */
-  public static boolean is(RexNode node, BitSet nullColumns) {
+  public static boolean is(RexNode node, ImmutableBitSet nullColumns) {
     return of(nullColumns).strong(node);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/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 0b95f82..c61208e 100644
--- a/core/src/main/java/org/apache/calcite/plan/SubstitutionVisitor.java
+++ b/core/src/main/java/org/apache/calcite/plan/SubstitutionVisitor.java
@@ -48,8 +48,8 @@ import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.validate.SqlValidatorUtil;
-import org.apache.calcite.util.BitSets;
 import org.apache.calcite.util.ControlFlowException;
+import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.IntList;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Util;
@@ -72,7 +72,6 @@ import com.google.common.collect.Sets;
 
 import java.util.AbstractList;
 import java.util.ArrayList;
-import java.util.BitSet;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
@@ -1134,7 +1133,7 @@ public class SubstitutionVisitor {
       //   target: SELECT x, y, SUM(a) AS s, COUNT(b) AS cb FROM t GROUP BY x, y
       // transforms to
       //   result: SELECT x, SUM(cb) FROM (target) GROUP BY x
-      if (!BitSets.contains(target.getGroupSet(), query.getGroupSet())) {
+      if (!target.getGroupSet().contains(query.getGroupSet())) {
         return null;
       }
       MutableRel result = unifyAggregates(query, target);
@@ -1147,7 +1146,7 @@ public class SubstitutionVisitor {
 
   public static MutableAggregate permute(MutableAggregate aggregate,
       MutableRel input, Mapping mapping) {
-    BitSet groupSet = Mappings.apply(mapping, aggregate.getGroupSet());
+    ImmutableBitSet groupSet = Mappings.apply(mapping, aggregate.getGroupSet());
     List<AggregateCall> aggregateCalls =
         apply(mapping, aggregate.getAggCallList());
     return MutableAggregate.of(input, groupSet, aggregateCalls);
@@ -1183,9 +1182,9 @@ public class SubstitutionVisitor {
       result = MutableRels.createProject(target, projects);
     } else {
       // Target is coarser level of aggregation. Generate an aggregate.
-      final BitSet groupSet = new BitSet();
-      final IntList targetGroupList = BitSets.toList(target.getGroupSet());
-      for (int c : BitSets.toIter(query.getGroupSet())) {
+      final ImmutableBitSet.Builder groupSet = ImmutableBitSet.builder();
+      final IntList targetGroupList = target.getGroupSet().toList();
+      for (int c : query.getGroupSet()) {
         int c2 = targetGroupList.indexOf(c);
         if (c2 < 0) {
           return null;
@@ -1207,7 +1206,7 @@ public class SubstitutionVisitor {
                 ImmutableList.of(target.groupSet.cardinality() + i),
                 aggregateCall.type, aggregateCall.name));
       }
-      result = MutableAggregate.of(target, groupSet, aggregateCalls);
+      result = MutableAggregate.of(target, groupSet.build(), aggregateCalls);
     }
     return MutableRels.createCastRel(result, query.getRowType(), true);
   }
@@ -1621,17 +1620,17 @@ public class SubstitutionVisitor {
   /** Mutable equivalent of
    * {@link org.apache.calcite.rel.logical.LogicalAggregate}. */
   private static class MutableAggregate extends MutableSingleRel {
-    private final BitSet groupSet;
+    private final ImmutableBitSet groupSet;
     private final List<AggregateCall> aggCalls;
 
     private MutableAggregate(MutableRel input, RelDataType rowType,
-        BitSet groupSet, List<AggregateCall> aggCalls) {
+        ImmutableBitSet groupSet, List<AggregateCall> aggCalls) {
       super(MutableRelType.AGGREGATE, rowType, input);
       this.groupSet = groupSet;
       this.aggCalls = aggCalls;
     }
 
-    static MutableAggregate of(MutableRel input, BitSet groupSet,
+    static MutableAggregate of(MutableRel input, ImmutableBitSet groupSet,
         List<AggregateCall> aggCalls) {
       RelDataType rowType =
           Aggregate.deriveRowType(input.cluster.getTypeFactory(),
@@ -1656,7 +1655,7 @@ public class SubstitutionVisitor {
           .append(", calls: ").append(aggCalls).append(")");
     }
 
-    public BitSet getGroupSet() {
+    public ImmutableBitSet getGroupSet() {
       return groupSet;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/core/src/main/java/org/apache/calcite/plan/hep/HepRelVertex.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/hep/HepRelVertex.java b/core/src/main/java/org/apache/calcite/plan/hep/HepRelVertex.java
index 5e8fab4..3ee9739 100644
--- a/core/src/main/java/org/apache/calcite/plan/hep/HepRelVertex.java
+++ b/core/src/main/java/org/apache/calcite/plan/hep/HepRelVertex.java
@@ -24,9 +24,9 @@ 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.ImmutableBitSet;
 import org.apache.calcite.util.Util;
 
-import java.util.BitSet;
 import java.util.List;
 
 /**
@@ -80,7 +80,7 @@ public class HepRelVertex extends AbstractRelNode {
     return currentRel.isDistinct();
   }
 
-  @Override public boolean isKey(BitSet columns) {
+  @Override public boolean isKey(ImmutableBitSet columns) {
     return currentRel.isKey(columns);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/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 294a29c..b08320b 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,13 +30,13 @@ 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.ImmutableBitSet;
 import org.apache.calcite.util.Util;
 import org.apache.calcite.util.trace.CalciteTrace;
 
 import java.io.PrintWriter;
 import java.io.StringWriter;
 import java.util.ArrayList;
-import java.util.BitSet;
 import java.util.Collection;
 import java.util.Iterator;
 import java.util.LinkedHashSet;
@@ -185,7 +185,7 @@ public class RelSubset extends AbstractRelNode {
     return false;
   }
 
-  @Override public boolean isKey(BitSet columns) {
+  @Override public boolean isKey(ImmutableBitSet columns) {
     for (RelNode rel : set.rels) {
       if (rel.isKey(columns)) {
         return true;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/core/src/main/java/org/apache/calcite/prepare/RelOptTableImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/prepare/RelOptTableImpl.java b/core/src/main/java/org/apache/calcite/prepare/RelOptTableImpl.java
index a3377a0..68e0dcf 100644
--- a/core/src/main/java/org/apache/calcite/prepare/RelOptTableImpl.java
+++ b/core/src/main/java/org/apache/calcite/prepare/RelOptTableImpl.java
@@ -35,6 +35,7 @@ import org.apache.calcite.schema.Table;
 import org.apache.calcite.schema.TranslatableTable;
 import org.apache.calcite.sql.SqlAccessType;
 import org.apache.calcite.sql.validate.SqlMonotonicity;
+import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.Util;
 
 import com.google.common.base.Function;
@@ -42,7 +43,6 @@ import com.google.common.base.Functions;
 import com.google.common.collect.ImmutableList;
 
 import java.lang.reflect.Type;
-import java.util.BitSet;
 import java.util.Collections;
 import java.util.List;
 
@@ -215,7 +215,7 @@ public class RelOptTableImpl implements Prepare.PreparingTable {
     return Collections.emptyList();
   }
 
-  public boolean isKey(BitSet columns) {
+  public boolean isKey(ImmutableBitSet columns) {
     return table.getStatistic().isKey(columns);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/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 4a60299..1bd85cb 100644
--- a/core/src/main/java/org/apache/calcite/rel/AbstractRelNode.java
+++ b/core/src/main/java/org/apache/calcite/rel/AbstractRelNode.java
@@ -32,7 +32,7 @@ import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.sql.SqlExplainLevel;
-import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Util;
 import org.apache.calcite.util.trace.CalciteTrace;
@@ -42,7 +42,6 @@ import com.google.common.collect.ImmutableList;
 import java.io.PrintWriter;
 import java.io.StringWriter;
 import java.util.ArrayList;
-import java.util.BitSet;
 import java.util.Collections;
 import java.util.List;
 import java.util.Set;
@@ -169,10 +168,10 @@ public abstract class AbstractRelNode implements RelNode {
   }
 
   public boolean isDistinct() {
-    return isKey(BitSets.range(getRowType().getFieldCount()));
+    return isKey(ImmutableBitSet.range(getRowType().getFieldCount()));
   }
 
-  public boolean isKey(BitSet columns) {
+  public boolean isKey(ImmutableBitSet columns) {
     return false;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/core/src/main/java/org/apache/calcite/rel/RelInput.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/RelInput.java b/core/src/main/java/org/apache/calcite/rel/RelInput.java
index 0eff922..2ffe64f 100644
--- a/core/src/main/java/org/apache/calcite/rel/RelInput.java
+++ b/core/src/main/java/org/apache/calcite/rel/RelInput.java
@@ -23,8 +23,8 @@ import org.apache.calcite.rel.core.AggregateCall;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.util.ImmutableBitSet;
 
-import java.util.BitSet;
 import java.util.List;
 
 /**
@@ -51,7 +51,7 @@ public interface RelInput {
    */
   RexNode getExpression(String tag);
 
-  BitSet getBitSet(String tag);
+  ImmutableBitSet getBitSet(String tag);
 
   List<AggregateCall> getAggregateCalls(String tag);
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/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 50da558..fa28861 100644
--- a/core/src/main/java/org/apache/calcite/rel/RelNode.java
+++ b/core/src/main/java/org/apache/calcite/rel/RelNode.java
@@ -27,8 +27,8 @@ import org.apache.calcite.rel.metadata.Metadata;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.util.ImmutableBitSet;
 
-import java.util.BitSet;
 import java.util.List;
 import java.util.Set;
 
@@ -345,7 +345,7 @@ public interface RelNode extends RelOptNode, Cloneable {
    * @param columns Ordinals of key columns
    * @return Whether the given columns are a key or a superset of a key
    */
-  boolean isKey(BitSet columns);
+  boolean isKey(ImmutableBitSet columns);
 
   /**
    * Accepts a visit from a shuttle.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/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 920ef46..d0e19be 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
@@ -38,15 +38,14 @@ import org.apache.calcite.sql.SqlOperatorBinding;
 import org.apache.calcite.sql.SqlUtil;
 import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.sql.validate.SqlValidatorException;
-import org.apache.calcite.util.BitSets;
 import org.apache.calcite.util.CompositeList;
+import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.IntList;
 import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
 
 import java.util.AbstractList;
-import java.util.BitSet;
 import java.util.List;
 
 /**
@@ -69,7 +68,7 @@ public abstract class Aggregate extends SingleRel {
   //~ Instance fields --------------------------------------------------------
 
   protected final List<AggregateCall> aggCalls;
-  protected final BitSet groupSet;
+  protected final ImmutableBitSet groupSet;
 
   //~ Constructors -----------------------------------------------------------
 
@@ -86,16 +85,13 @@ public abstract class Aggregate extends SingleRel {
       RelOptCluster cluster,
       RelTraitSet traits,
       RelNode child,
-      BitSet groupSet,
+      ImmutableBitSet groupSet,
       List<AggregateCall> aggCalls) {
     super(cluster, traits, child);
     assert aggCalls != null;
     this.aggCalls = ImmutableList.copyOf(aggCalls);
     this.groupSet = groupSet;
     assert groupSet != null;
-    assert groupSet.isEmpty() == (groupSet.cardinality() == 0)
-        : "See https://bugs.openjdk.java.net/browse/JDK-6222207, "
-        + "BitSet internal invariants may be violated";
     assert groupSet.length() <= child.getRowType().getFieldCount();
     for (AggregateCall aggCall : aggCalls) {
       assert typeMatchesInferred(aggCall, true);
@@ -122,7 +118,7 @@ public abstract class Aggregate extends SingleRel {
    * @see #copy(org.apache.calcite.plan.RelTraitSet, java.util.List)
    */
   public abstract Aggregate copy(RelTraitSet traitSet, RelNode input,
-      BitSet groupSet, List<AggregateCall> aggCalls);
+      ImmutableBitSet groupSet, List<AggregateCall> aggCalls);
 
   // implement RelNode
   public boolean isDistinct() {
@@ -156,11 +152,11 @@ public abstract class Aggregate extends SingleRel {
   }
 
   /**
-   * Returns a bitmap of the grouping fields.
+   * Returns a bit set of the grouping fields.
    *
-   * @return bitset of ordinals of grouping fields
+   * @return bit set of ordinals of grouping fields
    */
-  public BitSet getGroupSet() {
+  public ImmutableBitSet getGroupSet() {
     return groupSet;
   }
 
@@ -209,9 +205,9 @@ public abstract class Aggregate extends SingleRel {
 
   /** Computes the row type of an {@code Aggregate} before it exists. */
   public static RelDataType deriveRowType(RelDataTypeFactory typeFactory,
-      final RelDataType inputRowType, BitSet groupSet,
+      final RelDataType inputRowType, ImmutableBitSet groupSet,
       final List<AggregateCall> aggCalls) {
-    final IntList groupList = BitSets.toList(groupSet);
+    final IntList groupList = groupSet.toList();
     assert groupList.size() == groupSet.cardinality();
     return typeFactory.createStructType(
         CompositeList.of(
@@ -257,7 +253,7 @@ public abstract class Aggregate extends SingleRel {
   private boolean typeMatchesInferred(
       final AggregateCall aggCall,
       final boolean fail) {
-    SqlAggFunction aggFunction = (SqlAggFunction) aggCall.getAggregation();
+    SqlAggFunction aggFunction = aggCall.getAggregation();
     AggCallBinding callBinding = aggCall.createBinding(this);
     RelDataType type = aggFunction.inferReturnType(callBinding);
     RelDataType expectedType = aggCall.type;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/core/src/main/java/org/apache/calcite/rel/core/Intersect.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/Intersect.java b/core/src/main/java/org/apache/calcite/rel/core/Intersect.java
index 472cd92..139567c 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/Intersect.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/Intersect.java
@@ -22,8 +22,8 @@ import org.apache.calcite.rel.RelInput;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.util.ImmutableBitSet;
 
-import java.util.BitSet;
 import java.util.List;
 
 /**
@@ -63,7 +63,7 @@ public abstract class Intersect extends SetOp {
     return dRows;
   }
 
-  @Override public boolean isKey(BitSet columns) {
+  @Override public boolean isKey(ImmutableBitSet columns) {
     for (RelNode input : inputs) {
       if (input.isKey(columns)) {
         return true;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/core/src/main/java/org/apache/calcite/rel/core/JoinInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/JoinInfo.java b/core/src/main/java/org/apache/calcite/rel/core/JoinInfo.java
index b1c8a18..03a955e 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/JoinInfo.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/JoinInfo.java
@@ -20,14 +20,13 @@ import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rex.RexBuilder;
 import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.ImmutableIntList;
 import org.apache.calcite.util.mapping.IntPair;
 
 import com.google.common.base.Preconditions;
 
 import java.util.ArrayList;
-import java.util.BitSet;
 import java.util.List;
 
 /** An analyzed join condition.
@@ -82,12 +81,12 @@ public abstract class JoinInfo {
     return IntPair.zip(leftKeys, rightKeys);
   }
 
-  public BitSet leftSet() {
-    return BitSets.of(leftKeys);
+  public ImmutableBitSet leftSet() {
+    return ImmutableBitSet.of(leftKeys);
   }
 
-  public BitSet rightSet() {
-    return BitSets.of(rightKeys);
+  public ImmutableBitSet rightSet() {
+    return ImmutableBitSet.of(rightKeys);
   }
 
   public abstract RexNode getRemaining(RexBuilder rexBuilder);

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/core/src/main/java/org/apache/calcite/rel/core/Minus.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/Minus.java b/core/src/main/java/org/apache/calcite/rel/core/Minus.java
index 12feba8..92b0784 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/Minus.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/Minus.java
@@ -22,8 +22,8 @@ import org.apache.calcite.rel.RelInput;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.util.ImmutableBitSet;
 
-import java.util.BitSet;
 import java.util.List;
 
 /**
@@ -61,7 +61,7 @@ public abstract class Minus extends SetOp {
     return dRows;
   }
 
-  @Override public boolean isKey(BitSet columns) {
+  @Override public boolean isKey(ImmutableBitSet columns) {
     return inputs.get(0).isKey(columns)
         || super.isKey(columns);
   }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/core/src/main/java/org/apache/calcite/rel/core/RelFactories.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/RelFactories.java b/core/src/main/java/org/apache/calcite/rel/core/RelFactories.java
index fcc9a2d..332fdd5 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/RelFactories.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/RelFactories.java
@@ -31,10 +31,10 @@ import org.apache.calcite.rel.logical.LogicalUnion;
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.util.ImmutableBitSet;
 
 import com.google.common.collect.ImmutableList;
 
-import java.util.BitSet;
 import java.util.List;
 import java.util.Set;
 
@@ -148,7 +148,7 @@ public class RelFactories {
    */
   public interface AggregateFactory {
     /** Creates an aggregate. */
-    RelNode createAggregate(RelNode child, BitSet groupSet,
+    RelNode createAggregate(RelNode child, ImmutableBitSet groupSet,
         List<AggregateCall> aggCalls);
   }
 
@@ -157,7 +157,7 @@ public class RelFactories {
    * that returns a vanilla {@link LogicalAggregate}.
    */
   private static class AggregateFactoryImpl implements AggregateFactory {
-    public RelNode createAggregate(RelNode child, BitSet groupSet,
+    public RelNode createAggregate(RelNode child, ImmutableBitSet groupSet,
         List<AggregateCall> aggCalls) {
       return new LogicalAggregate(child.getCluster(), child, groupSet,
           aggCalls);

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/core/src/main/java/org/apache/calcite/rel/core/SetOp.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/SetOp.java b/core/src/main/java/org/apache/calcite/rel/core/SetOp.java
index 7b0187a..2b17733 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/SetOp.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/SetOp.java
@@ -26,13 +26,13 @@ import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.RelWriter;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.util.ImmutableBitSet;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 
 import java.util.AbstractList;
 import java.util.ArrayList;
-import java.util.BitSet;
 import java.util.List;
 
 /**
@@ -88,7 +88,7 @@ public abstract class SetOp extends AbstractRelNode {
     recomputeDigest();
   }
 
-  @Override public boolean isKey(BitSet columns) {
+  @Override public boolean isKey(ImmutableBitSet columns) {
     // If not ALL then the rows are distinct.
     // Therefore the set of all columns is a key.
     return !all && columns.nextClearBit(0) >= getRowType().getFieldCount();

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/core/src/main/java/org/apache/calcite/rel/core/TableScan.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/TableScan.java b/core/src/main/java/org/apache/calcite/rel/core/TableScan.java
index 9eaa931..496e39b 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/TableScan.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/TableScan.java
@@ -31,10 +31,9 @@ import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rex.RexBuilder;
 import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.ImmutableBitSet;
 
 import java.util.ArrayList;
-import java.util.BitSet;
 import java.util.List;
 import java.util.Set;
 
@@ -83,7 +82,7 @@ public abstract class TableScan extends AbstractRelNode {
     return table.getCollationList();
   }
 
-  @Override public boolean isKey(BitSet columns) {
+  @Override public boolean isKey(ImmutableBitSet columns) {
     return table.isKey(columns);
   }
 
@@ -120,10 +119,11 @@ public abstract class TableScan extends AbstractRelNode {
    *                    wanted by the consumer
    * @return Relational expression that projects the desired fields
    */
-  public RelNode project(BitSet fieldsUsed, Set<RelDataTypeField> extraFields,
-    RelFactories.ProjectFactory projectFactory) {
+  public RelNode project(ImmutableBitSet fieldsUsed,
+      Set<RelDataTypeField> extraFields,
+      RelFactories.ProjectFactory projectFactory) {
     final int fieldCount = getRowType().getFieldCount();
-    if (fieldsUsed.equals(BitSets.range(fieldCount))
+    if (fieldsUsed.equals(ImmutableBitSet.range(fieldCount))
         && extraFields.isEmpty()) {
       return this;
     }
@@ -133,7 +133,7 @@ public abstract class TableScan extends AbstractRelNode {
     final List<RelDataTypeField> fields = getRowType().getFieldList();
 
     // Project the subset of fields.
-    for (int i : BitSets.toIter(fieldsUsed)) {
+    for (int i : fieldsUsed) {
       RelDataTypeField field = fields.get(i);
       exprList.add(rexBuilder.makeInputRef(this, i));
       nameList.add(field.getName());

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/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 043da29..fc090f8 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
@@ -35,13 +35,13 @@ import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexSlot;
 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.Util;
 
 import com.google.common.collect.ImmutableList;
 
 import java.util.AbstractList;
-import java.util.BitSet;
 import java.util.List;
 
 /**
@@ -190,7 +190,7 @@ public abstract class Window extends SingleRel {
    * </ul>
    */
   public static class Group {
-    public final BitSet keys;
+    public final ImmutableBitSet keys;
     public final boolean isRows;
     public final RexWindowBound lowerBound;
     public final RexWindowBound upperBound;
@@ -205,7 +205,7 @@ public abstract class Window extends SingleRel {
     public final ImmutableList<RexWinAggCall> aggCalls;
 
     public Group(
-        BitSet keys,
+        ImmutableBitSet keys,
         boolean isRows,
         RexWindowBound lowerBound,
         RexWindowBound upperBound,

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/core/src/main/java/org/apache/calcite/rel/externalize/RelJson.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/externalize/RelJson.java b/core/src/main/java/org/apache/calcite/rel/externalize/RelJson.java
index 4c77805..ff6c570 100644
--- a/core/src/main/java/org/apache/calcite/rel/externalize/RelJson.java
+++ b/core/src/main/java/org/apache/calcite/rel/externalize/RelJson.java
@@ -39,7 +39,7 @@ import org.apache.calcite.sql.SqlFunction;
 import org.apache.calcite.sql.SqlOperator;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.type.SqlTypeName;
-import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.JsonBuilder;
 import org.apache.calcite.util.Util;
 
@@ -49,7 +49,6 @@ import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
 import java.math.BigDecimal;
 import java.util.ArrayList;
-import java.util.BitSet;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -211,7 +210,7 @@ public class RelJson {
 
   public Object toJson(AggregateCall node) {
     final Map<String, Object> map = jsonBuilder.map();
-    map.put("agg", toJson((SqlOperator) node.getAggregation()));
+    map.put("agg", toJson(node.getAggregation()));
     map.put("type", toJson(node.getType()));
     map.put("distinct", node.isDistinct());
     map.put("operands", node.getArgList());
@@ -234,9 +233,9 @@ public class RelJson {
         list.add(toJson(o));
       }
       return list;
-    } else if (value instanceof BitSet) {
+    } else if (value instanceof ImmutableBitSet) {
       final List<Object> list = jsonBuilder.list();
-      for (Integer integer : BitSets.toIter((BitSet) value)) {
+      for (Integer integer : (ImmutableBitSet) value) {
         list.add(toJson(integer));
       }
       return list;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/core/src/main/java/org/apache/calcite/rel/externalize/RelJsonReader.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/externalize/RelJsonReader.java b/core/src/main/java/org/apache/calcite/rel/externalize/RelJsonReader.java
index 8e20b7f..31dbdac 100644
--- a/core/src/main/java/org/apache/calcite/rel/externalize/RelJsonReader.java
+++ b/core/src/main/java/org/apache/calcite/rel/externalize/RelJsonReader.java
@@ -30,7 +30,7 @@ import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.schema.Schema;
 import org.apache.calcite.sql.SqlAggFunction;
-import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Util;
 
@@ -44,7 +44,6 @@ import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
 import java.util.AbstractList;
 import java.util.ArrayList;
-import java.util.BitSet;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
@@ -131,8 +130,8 @@ public class RelJsonReader {
         return relJson.toRex(this, jsonRel.get(tag));
       }
 
-      public BitSet getBitSet(String tag) {
-        return BitSets.of(getIntegerList(tag));
+      public ImmutableBitSet getBitSet(String tag) {
+        return ImmutableBitSet.of(getIntegerList(tag));
       }
 
       public List<Integer> getIntegerList(String tag) {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/core/src/main/java/org/apache/calcite/rel/logical/LogicalAggregate.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/logical/LogicalAggregate.java b/core/src/main/java/org/apache/calcite/rel/logical/LogicalAggregate.java
index 198f242..5f1d579 100644
--- a/core/src/main/java/org/apache/calcite/rel/logical/LogicalAggregate.java
+++ b/core/src/main/java/org/apache/calcite/rel/logical/LogicalAggregate.java
@@ -24,8 +24,8 @@ import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.RelShuttle;
 import org.apache.calcite.rel.core.Aggregate;
 import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.util.ImmutableBitSet;
 
-import java.util.BitSet;
 import java.util.List;
 
 /**
@@ -54,7 +54,7 @@ public final class LogicalAggregate extends Aggregate {
   public LogicalAggregate(
       RelOptCluster cluster,
       RelNode child,
-      BitSet groupSet,
+      ImmutableBitSet groupSet,
       List<AggregateCall> aggCalls) {
     super(
         cluster,
@@ -74,7 +74,7 @@ public final class LogicalAggregate extends Aggregate {
   //~ Methods ----------------------------------------------------------------
 
   @Override public LogicalAggregate copy(RelTraitSet traitSet, RelNode input,
-      BitSet groupSet, List<AggregateCall> aggCalls) {
+      ImmutableBitSet groupSet, List<AggregateCall> aggCalls) {
     assert traitSet.containsIfApplicable(Convention.NONE);
     return new LogicalAggregate(getCluster(), input, groupSet, aggCalls);
   }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/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 e75d340..4534d56 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
@@ -34,7 +34,7 @@ import org.apache.calcite.rex.RexProgram;
 import org.apache.calcite.rex.RexShuttle;
 import org.apache.calcite.rex.RexWindow;
 import org.apache.calcite.rex.RexWindowBound;
-import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.Pair;
 
 import com.google.common.base.Objects;
@@ -46,7 +46,6 @@ import com.google.common.collect.Multimap;
 
 import java.util.AbstractList;
 import java.util.ArrayList;
-import java.util.BitSet;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.List;
@@ -275,14 +274,14 @@ public final class LogicalWindow extends Window {
    * (regardless of how it is specified, in terms of a named window or specified
    * attribute by attribute) will end up with the same window key. */
   private static class WindowKey {
-    private final BitSet groupSet;
+    private final ImmutableBitSet groupSet;
     private final RelCollation orderKeys;
     private final boolean isRows;
     private final RexWindowBound lowerBound;
     private final RexWindowBound upperBound;
 
     public WindowKey(
-        BitSet groupSet,
+        ImmutableBitSet groupSet,
         RelCollation orderKeys,
         boolean isRows,
         RexWindowBound lowerBound,
@@ -329,13 +328,14 @@ public final class LogicalWindow extends Window {
               return rexFieldCollation.left instanceof RexLocalRef;
             }
           })));
-    BitSet groupSet =
-        BitSets.of(getProjectOrdinals(aggWindow.partitionKeys));
+    ImmutableBitSet groupSet =
+        ImmutableBitSet.of(getProjectOrdinals(aggWindow.partitionKeys));
     final int groupLength = groupSet.length();
     if (inputFieldCount < groupLength) {
       // If PARTITION BY references constant, we can ignore such partition key.
       // All the inputs after inputFieldCount are literals, thus we can clear.
-      groupSet.clear(inputFieldCount, groupLength);
+      groupSet =
+          groupSet.except(ImmutableBitSet.range(inputFieldCount, groupLength));
     }
 
     WindowKey windowKey =

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/core/src/main/java/org/apache/calcite/rel/metadata/BuiltInMetadata.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/BuiltInMetadata.java b/core/src/main/java/org/apache/calcite/rel/metadata/BuiltInMetadata.java
index 4d890b6..5aa668f 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/BuiltInMetadata.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/BuiltInMetadata.java
@@ -21,8 +21,8 @@ import org.apache.calcite.plan.RelOptPredicateList;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.util.ImmutableBitSet;
 
-import java.util.BitSet;
 import java.util.Set;
 
 /**
@@ -49,8 +49,8 @@ public abstract class BuiltInMetadata {
   public interface UniqueKeys extends Metadata {
     /**
      * Determines the set of unique minimal keys for this expression. A key is
-     * represented as a {@link BitSet}, where each bit position represents a
-     * 0-based output column ordinal.
+     * represented as an {@link org.apache.calcite.util.ImmutableBitSet}, where
+     * each bit position represents a 0-based output column ordinal.
      *
      * <p>Note that {@link RelNode#isDistinct} should return {@code true} if and
      * only if at least one key is known.</p>
@@ -63,7 +63,7 @@ public abstract class BuiltInMetadata {
      * @return set of keys, or null if this information cannot be determined
      * (whereas empty set indicates definitely no keys at all)
      */
-    Set<BitSet> getUniqueKeys(boolean ignoreNulls);
+    Set<ImmutableBitSet> getUniqueKeys(boolean ignoreNulls);
   }
 
   /** Metadata about whether a set of columns uniquely identifies a row. */
@@ -82,7 +82,7 @@ public abstract class BuiltInMetadata {
      * @return whether the columns are unique, or
      * null if not enough information is available to make that determination
      */
-    Boolean areColumnsUnique(BitSet columns, boolean ignoreNulls);
+    Boolean areColumnsUnique(ImmutableBitSet columns, boolean ignoreNulls);
   }
 
   /** Metadata about the number of rows returned by a relational expression. */
@@ -114,7 +114,7 @@ public abstract class BuiltInMetadata {
      * @return distinct row count for groupKey, filtered by predicate, or null
      * if no reliable estimate can be determined
      */
-    Double getDistinctRowCount(BitSet groupKey, RexNode predicate);
+    Double getDistinctRowCount(ImmutableBitSet groupKey, RexNode predicate);
   }
 
   /** Metadata about the proportion of original rows that remain in a relational
@@ -145,7 +145,7 @@ public abstract class BuiltInMetadata {
      * @return distinct row count for the given groupKey, or null if no reliable
      * estimate can be determined
      */
-    Double getPopulationSize(BitSet groupKey);
+    Double getPopulationSize(ImmutableBitSet groupKey);
   }
 
   /** Metadata about the origins of columns. */

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/core/src/main/java/org/apache/calcite/rel/metadata/RelMdColumnUniqueness.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdColumnUniqueness.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdColumnUniqueness.java
index 262d9e5..bede712 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdColumnUniqueness.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdColumnUniqueness.java
@@ -31,10 +31,9 @@ import org.apache.calcite.rex.RexCall;
 import org.apache.calcite.rex.RexInputRef;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
-import org.apache.calcite.util.BitSets;
 import org.apache.calcite.util.BuiltInMethod;
+import org.apache.calcite.util.ImmutableBitSet;
 
-import java.util.BitSet;
 import java.util.List;
 
 /**
@@ -54,7 +53,7 @@ public class RelMdColumnUniqueness {
 
   public Boolean areColumnsUnique(
       Filter rel,
-      BitSet columns,
+      ImmutableBitSet columns,
       boolean ignoreNulls) {
     return RelMetadataQuery.areColumnsUnique(
         rel.getInput(),
@@ -64,7 +63,7 @@ public class RelMdColumnUniqueness {
 
   public Boolean areColumnsUnique(
       Sort rel,
-      BitSet columns,
+      ImmutableBitSet columns,
       boolean ignoreNulls) {
     return RelMetadataQuery.areColumnsUnique(
         rel.getInput(),
@@ -74,7 +73,7 @@ public class RelMdColumnUniqueness {
 
   public Boolean areColumnsUnique(
       Correlator rel,
-      BitSet columns,
+      ImmutableBitSet columns,
       boolean ignoreNulls) {
     return RelMetadataQuery.areColumnsUnique(
         rel.getLeft(),
@@ -84,7 +83,7 @@ public class RelMdColumnUniqueness {
 
   public Boolean areColumnsUnique(
       Project rel,
-      BitSet columns,
+      ImmutableBitSet columns,
       boolean ignoreNulls) {
     // LogicalProject maps a set of rows to a different set;
     // Without knowledge of the mapping function(whether it
@@ -95,8 +94,8 @@ public class RelMdColumnUniqueness {
     // references
 
     List<RexNode> projExprs = rel.getProjects();
-    BitSet childColumns = new BitSet();
-    for (int bit : BitSets.toIter(columns)) {
+    ImmutableBitSet.Builder childColumns = ImmutableBitSet.builder();
+    for (int bit : columns) {
       RexNode projExpr = projExprs.get(bit);
       if (projExpr instanceof RexInputRef) {
         childColumns.set(((RexInputRef) projExpr).getIndex());
@@ -139,14 +138,14 @@ public class RelMdColumnUniqueness {
 
     return RelMetadataQuery.areColumnsUnique(
         rel.getInput(),
-        childColumns,
+        childColumns.build(),
         ignoreNulls);
   }
 
   public Boolean areColumnsUnique(
       Join rel,
-      BitSet columns, boolean
-      ignoreNulls) {
+      ImmutableBitSet columns,
+      boolean ignoreNulls) {
     if (columns.cardinality() == 0) {
       return false;
     }
@@ -156,10 +155,10 @@ public class RelMdColumnUniqueness {
 
     // Divide up the input column mask into column masks for the left and
     // right sides of the join
-    BitSet leftColumns = new BitSet();
-    BitSet rightColumns = new BitSet();
+    ImmutableBitSet.Builder leftColumns = ImmutableBitSet.builder();
+    ImmutableBitSet.Builder rightColumns = ImmutableBitSet.builder();
     int nLeftColumns = left.getRowType().getFieldCount();
-    for (int bit : BitSets.toIter(columns)) {
+    for (int bit : columns) {
       if (bit < nLeftColumns) {
         leftColumns.set(bit);
       } else {
@@ -171,9 +170,11 @@ public class RelMdColumnUniqueness {
     // right hand side, then the columns are unique if and only if they're
     // unique for their respective join inputs
     Boolean leftUnique =
-        RelMetadataQuery.areColumnsUnique(left, leftColumns, ignoreNulls);
+        RelMetadataQuery.areColumnsUnique(left, leftColumns.build(),
+            ignoreNulls);
     Boolean rightUnique =
-        RelMetadataQuery.areColumnsUnique(right, rightColumns, ignoreNulls);
+        RelMetadataQuery.areColumnsUnique(right, rightColumns.build(),
+            ignoreNulls);
     if ((leftColumns.cardinality() > 0)
         && (rightColumns.cardinality() > 0)) {
       if ((leftUnique == null) || (rightUnique == null)) {
@@ -219,7 +220,7 @@ public class RelMdColumnUniqueness {
 
   public Boolean areColumnsUnique(
       SemiJoin rel,
-      BitSet columns,
+      ImmutableBitSet columns,
       boolean ignoreNulls) {
     // only return the unique keys from the LHS since a semijoin only
     // returns the LHS
@@ -231,15 +232,12 @@ public class RelMdColumnUniqueness {
 
   public Boolean areColumnsUnique(
       Aggregate rel,
-      BitSet columns,
+      ImmutableBitSet columns,
       boolean ignoreNulls) {
     // group by keys form a unique key
     if (rel.getGroupCount() > 0) {
-      BitSet groupKey = new BitSet();
-      for (int i = 0; i < rel.getGroupCount(); i++) {
-        groupKey.set(i);
-      }
-      return BitSets.contains(columns, groupKey);
+      ImmutableBitSet groupKey = ImmutableBitSet.range(rel.getGroupCount());
+      return columns.contains(groupKey);
     } else {
       // interpret an empty set as asking whether the aggregation is full
       // table (in which case it returns at most one row);
@@ -253,7 +251,7 @@ public class RelMdColumnUniqueness {
   // Catch-all rule when none of the others apply.
   public Boolean areColumnsUnique(
       RelNode rel,
-      BitSet columns,
+      ImmutableBitSet columns,
       boolean ignoreNulls) {
     // no information available
     return null;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/core/src/main/java/org/apache/calcite/rel/metadata/RelMdDistinctRowCount.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdDistinctRowCount.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdDistinctRowCount.java
index e4d847d..2092379 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdDistinctRowCount.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdDistinctRowCount.java
@@ -30,12 +30,11 @@ import org.apache.calcite.rex.RexBuilder;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
-import org.apache.calcite.util.BitSets;
 import org.apache.calcite.util.BuiltInMethod;
+import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.NumberUtil;
 
 import java.util.ArrayList;
-import java.util.BitSet;
 import java.util.List;
 
 /**
@@ -56,7 +55,7 @@ public class RelMdDistinctRowCount {
 
   public Double getDistinctRowCount(
       Union rel,
-      BitSet groupKey,
+      ImmutableBitSet groupKey,
       RexNode predicate) {
     Double rowCount = 0.0;
     int[] adjustments = new int[rel.getRowType().getFieldCount()];
@@ -90,7 +89,7 @@ public class RelMdDistinctRowCount {
 
   public Double getDistinctRowCount(
       Sort rel,
-      BitSet groupKey,
+      ImmutableBitSet groupKey,
       RexNode predicate) {
     return RelMetadataQuery.getDistinctRowCount(
         rel.getInput(),
@@ -100,7 +99,7 @@ public class RelMdDistinctRowCount {
 
   public Double getDistinctRowCount(
       Filter rel,
-      BitSet groupKey,
+      ImmutableBitSet groupKey,
       RexNode predicate) {
     // REVIEW zfong 4/18/06 - In the Broadbase code, duplicates are not
     // removed from the two filter lists.  However, the code below is
@@ -119,7 +118,7 @@ public class RelMdDistinctRowCount {
 
   public Double getDistinctRowCount(
       Join rel,
-      BitSet groupKey,
+      ImmutableBitSet groupKey,
       RexNode predicate) {
     return RelMdUtil.getJoinDistinctRowCount(
         rel,
@@ -131,7 +130,7 @@ public class RelMdDistinctRowCount {
 
   public Double getDistinctRowCount(
       SemiJoin rel,
-      BitSet groupKey,
+      ImmutableBitSet groupKey,
       RexNode predicate) {
     // create a RexNode representing the selectivity of the
     // semijoin filter and pass it to getDistinctRowCount
@@ -153,7 +152,7 @@ public class RelMdDistinctRowCount {
 
   public Double getDistinctRowCount(
       Aggregate rel,
-      BitSet groupKey,
+      ImmutableBitSet groupKey,
       RexNode predicate) {
     // determine which predicates can be applied on the child of the
     // aggregate
@@ -169,13 +168,13 @@ public class RelMdDistinctRowCount {
         RexUtil.composeConjunction(rexBuilder, pushable, true);
 
     // set the bits as they correspond to the child input
-    BitSet childKey = new BitSet();
+    ImmutableBitSet.Builder childKey = ImmutableBitSet.builder();
     RelMdUtil.setAggChildKeys(groupKey, rel, childKey);
 
     Double distinctRowCount =
         RelMetadataQuery.getDistinctRowCount(
             rel.getInput(),
-            childKey,
+            childKey.build(),
             childPreds);
     if (distinctRowCount == null) {
       return null;
@@ -190,7 +189,7 @@ public class RelMdDistinctRowCount {
 
   public Double getDistinctRowCount(
       Values rel,
-      BitSet groupKey,
+      ImmutableBitSet groupKey,
       RexNode predicate) {
     Double selectivity = RelMdUtil.guessSelectivity(predicate);
 
@@ -201,17 +200,17 @@ public class RelMdDistinctRowCount {
 
   public Double getDistinctRowCount(
       Project rel,
-      BitSet groupKey,
+      ImmutableBitSet groupKey,
       RexNode predicate) {
-    BitSet baseCols = new BitSet();
-    BitSet projCols = new BitSet();
+    ImmutableBitSet.Builder baseCols = ImmutableBitSet.builder();
+    ImmutableBitSet.Builder projCols = ImmutableBitSet.builder();
     List<RexNode> projExprs = rel.getProjects();
     RelMdUtil.splitCols(projExprs, groupKey, baseCols, projCols);
 
     List<RexNode> notPushable = new ArrayList<RexNode>();
     List<RexNode> pushable = new ArrayList<RexNode>();
     RelOptUtil.splitFilters(
-        BitSets.range(rel.getRowType().getFieldCount()),
+        ImmutableBitSet.range(rel.getRowType().getFieldCount()),
         predicate,
         pushable,
         notPushable);
@@ -231,7 +230,7 @@ public class RelMdDistinctRowCount {
     Double distinctRowCount =
         RelMetadataQuery.getDistinctRowCount(
             rel.getInput(),
-            baseCols,
+            baseCols.build(),
             modifiedPred);
 
     if (distinctRowCount == null) {
@@ -249,7 +248,7 @@ public class RelMdDistinctRowCount {
     }
 
     // multiply by the cardinality of the non-child projection expressions
-    for (int bit : BitSets.toIter(projCols)) {
+    for (int bit : projCols.build()) {
       Double subRowCount =
           RelMdUtil.cardOfProjExpr(rel, projExprs.get(bit));
       if (subRowCount == null) {
@@ -266,7 +265,7 @@ public class RelMdDistinctRowCount {
   // Catch-all rule when none of the others apply.
   public Double getDistinctRowCount(
       RelNode rel,
-      BitSet groupKey,
+      ImmutableBitSet groupKey,
       RexNode predicate) {
     // REVIEW zfong 4/19/06 - Broadbase code does not take into
     // consideration selectivity of predicates passed in.  Also, they


[36/58] [abbrv] [partial] incubator-calcite git commit: [CALCITE-306] Standardize code style for "import package.*; "

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/externalize/RelJson.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/externalize/RelJson.java b/core/src/main/java/org/apache/calcite/rel/externalize/RelJson.java
index 1b4d76f..4c77805 100644
--- a/core/src/main/java/org/apache/calcite/rel/externalize/RelJson.java
+++ b/core/src/main/java/org/apache/calcite/rel/externalize/RelJson.java
@@ -14,29 +14,49 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel;
+package org.apache.calcite.rel.externalize;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollationImpl;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.RelInput;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.core.Correlation;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexCorrelVariable;
+import org.apache.calcite.rex.RexFieldAccess;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.SqlAggFunction;
+import org.apache.calcite.sql.SqlFunction;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.JsonBuilder;
+import org.apache.calcite.util.Util;
+
+import com.google.common.collect.ImmutableList;
 
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
 import java.math.BigDecimal;
-import java.util.*;
-
-import org.eigenbase.relopt.RelOptCluster;
-import org.eigenbase.reltype.*;
-import org.eigenbase.rex.*;
-import org.eigenbase.sql.SqlFunction;
-import org.eigenbase.sql.SqlOperator;
-import org.eigenbase.sql.fun.SqlStdOperatorTable;
-import org.eigenbase.sql.type.SqlTypeName;
-import org.eigenbase.util.JsonBuilder;
-import org.eigenbase.util.Util;
-
-import net.hydromatic.optiq.util.BitSets;
-
-import com.google.common.collect.ImmutableList;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
 
 /**
- * Utilities for converting {@link RelNode} into JSON format.
+ * Utilities for converting {@link org.apache.calcite.rel.RelNode}
+ * into JSON format.
  */
 public class RelJson {
   private final Map<String, Constructor> constructorMap =
@@ -45,9 +65,11 @@ public class RelJson {
 
   public static final List<String> PACKAGES =
       ImmutableList.of(
-          "org.eigenbase.rel.",
-          "net.hydromatic.optiq.impl.jdbc.",
-          "net.hydromatic.optiq.impl.jdbc.JdbcRules$");
+          "org.apache.calcite.rel.",
+          "org.apache.calcite.rel.core.",
+          "org.apache.calcite.rel.logical.",
+          "org.apache.calcite.adapter.jdbc.",
+          "org.apache.calcite.adapter.jdbc.JdbcRules$");
 
   public RelJson(JsonBuilder jsonBuilder) {
     this.jsonBuilder = jsonBuilder;
@@ -81,9 +103,8 @@ public class RelJson {
         //noinspection unchecked
         constructor = clazz.getConstructor(RelInput.class);
       } catch (NoSuchMethodException e) {
-        throw new RuntimeException(
-            "class does not have required constructor, " + clazz
-            + "(RelInput)");
+        throw new RuntimeException("class does not have required constructor, "
+            + clazz + "(RelInput)");
       }
       constructorMap.put(type, constructor);
     }
@@ -91,8 +112,8 @@ public class RelJson {
   }
 
   /**
-   * Converts a type name to a class. E.g. {@code getClass("ProjectRel")}
-   * returns {@link org.eigenbase.rel.ProjectRel}.class.
+   * Converts a type name to a class. E.g. {@code getClass("LogicalProject")}
+   * returns {@link org.apache.calcite.rel.logical.LogicalProject}.class.
    */
   public Class typeNameToClass(String type) {
     if (!type.contains(".")) {
@@ -436,8 +457,8 @@ public class RelJson {
     return null;
   }
 
-  Aggregation toAggregation(String agg, Map<String, Object> map) {
-    return (Aggregation) toOp(agg, map);
+  SqlAggFunction toAggregation(String agg, Map<String, Object> map) {
+    return (SqlAggFunction) toOp(agg, map);
   }
 
   private String toJson(SqlOperator operator) {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/externalize/RelJsonReader.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/externalize/RelJsonReader.java b/core/src/main/java/org/apache/calcite/rel/externalize/RelJsonReader.java
index b99dd1f..8e20b7f 100644
--- a/core/src/main/java/org/apache/calcite/rel/externalize/RelJsonReader.java
+++ b/core/src/main/java/org/apache/calcite/rel/externalize/RelJsonReader.java
@@ -14,32 +14,45 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel;
-
-import java.io.IOException;
-import java.lang.reflect.Constructor;
-import java.lang.reflect.InvocationTargetException;
-import java.util.*;
-
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.rex.*;
-import org.eigenbase.util.Pair;
-import org.eigenbase.util.Util;
-
-import net.hydromatic.optiq.Schema;
-import net.hydromatic.optiq.util.BitSets;
+package org.apache.calcite.rel.externalize;
+
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptSchema;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelInput;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.schema.Schema;
+import org.apache.calcite.sql.SqlAggFunction;
+import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
 
 import com.fasterxml.jackson.core.JsonParser;
 import com.fasterxml.jackson.core.type.TypeReference;
 import com.fasterxml.jackson.databind.ObjectMapper;
-
 import com.google.common.collect.ImmutableList;
 
+import java.io.IOException;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
+import java.util.AbstractList;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+
 /**
  * Reads a JSON plan and converts it back to a tree of relational expressions.
  *
- * @see org.eigenbase.rel.RelInput
+ * @see org.apache.calcite.rel.RelInput
  */
 public class RelJsonReader {
   private static final TypeReference<LinkedHashMap<String, Object>> TYPE_REF =
@@ -175,14 +188,12 @@ public class RelJsonReader {
             (List<String>) get(fieldsTag);
         return cluster.getTypeFactory().createStructType(
             new AbstractList<Map.Entry<String, RelDataType>>() {
-              @Override
-              public Map.Entry<String, RelDataType> get(int index) {
+              @Override public Map.Entry<String, RelDataType> get(int index) {
                 return Pair.of(names.get(index),
                     expressionList.get(index).getType());
               }
 
-              @Override
-              public int size() {
+              @Override public int size() {
                 return names.size();
               }
             });
@@ -228,7 +239,8 @@ public class RelJsonReader {
 
   private AggregateCall toAggCall(Map<String, Object> jsonAggCall) {
     final String aggName = (String) jsonAggCall.get("agg");
-    final Aggregation aggregation = relJson.toAggregation(aggName, jsonAggCall);
+    final SqlAggFunction aggregation =
+        relJson.toAggregation(aggName, jsonAggCall);
     final Boolean distinct = (Boolean) jsonAggCall.get("distinct");
     final List<Integer> operands = (List<Integer>) jsonAggCall.get("operands");
     final RelDataType type =

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/externalize/RelJsonWriter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/externalize/RelJsonWriter.java b/core/src/main/java/org/apache/calcite/rel/externalize/RelJsonWriter.java
index 4bef2ef..28981c8 100644
--- a/core/src/main/java/org/apache/calcite/rel/externalize/RelJsonWriter.java
+++ b/core/src/main/java/org/apache/calcite/rel/externalize/RelJsonWriter.java
@@ -14,19 +14,25 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel;
+package org.apache.calcite.rel.externalize;
 
-import java.util.*;
-
-import org.eigenbase.sql.SqlExplainLevel;
-import org.eigenbase.util.*;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelWriter;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.util.JsonBuilder;
+import org.apache.calcite.util.Pair;
 
 import com.google.common.collect.ImmutableList;
 
+import java.util.ArrayList;
+import java.util.IdentityHashMap;
+import java.util.List;
+import java.util.Map;
+
 /**
  * Callback for a relational expression to dump itself as JSON.
  *
- * @see org.eigenbase.rel.RelJsonReader
+ * @see RelJsonReader
  */
 public class RelJsonWriter implements RelWriter {
   //~ Instance fields ----------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/externalize/RelWriterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/externalize/RelWriterImpl.java b/core/src/main/java/org/apache/calcite/rel/externalize/RelWriterImpl.java
index 7262d56..0bf5ffb 100644
--- a/core/src/main/java/org/apache/calcite/rel/externalize/RelWriterImpl.java
+++ b/core/src/main/java/org/apache/calcite/rel/externalize/RelWriterImpl.java
@@ -14,24 +14,25 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel;
+package org.apache.calcite.rel.externalize;
 
-import java.io.PrintWriter;
-import java.util.*;
-
-import org.eigenbase.rel.metadata.*;
-import org.eigenbase.rex.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.util.*;
-
-import net.hydromatic.linq4j.Ord;
-
-import net.hydromatic.optiq.runtime.Spacer;
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelWriter;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.runtime.Spacer;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.util.Pair;
 
 import com.google.common.collect.ImmutableList;
 
+import java.io.PrintWriter;
+import java.util.ArrayList;
+import java.util.List;
+
 /**
- * Implementation of {@link org.eigenbase.rel.RelWriter}.
+ * Implementation of {@link org.apache.calcite.rel.RelWriter}.
  */
 public class RelWriterImpl implements RelWriter {
   //~ Instance fields --------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/externalize/RelXmlWriter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/externalize/RelXmlWriter.java b/core/src/main/java/org/apache/calcite/rel/externalize/RelXmlWriter.java
index 2108f6b..50e5381 100644
--- a/core/src/main/java/org/apache/calcite/rel/externalize/RelXmlWriter.java
+++ b/core/src/main/java/org/apache/calcite/rel/externalize/RelXmlWriter.java
@@ -14,14 +14,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel;
+package org.apache.calcite.rel.externalize;
 
-import java.io.*;
-import java.util.*;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.XmlOutput;
 
-import org.eigenbase.sql.*;
-import org.eigenbase.util.Pair;
-import org.eigenbase.util.XmlOutput;
+import java.io.PrintWriter;
+import java.util.ArrayList;
+import java.util.List;
 
 /**
  * Callback for a relational expression to dump in XML format.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/externalize/package-info.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/externalize/package-info.java b/core/src/main/java/org/apache/calcite/rel/externalize/package-info.java
new file mode 100644
index 0000000..d5897ff
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/rel/externalize/package-info.java
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Facilities to externalize {@link org.apache.calcite.rel.RelNode}s to and from
+ * XML and JSON format.
+ */
+package org.apache.calcite.rel.externalize;
+
+// End package-info.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/jdbc/package-info.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/jdbc/package-info.java b/core/src/main/java/org/apache/calcite/rel/jdbc/package-info.java
index c851a88..877c848 100644
--- a/core/src/main/java/org/apache/calcite/rel/jdbc/package-info.java
+++ b/core/src/main/java/org/apache/calcite/rel/jdbc/package-info.java
@@ -19,6 +19,6 @@
  * Contains query transformation rules relating to generating SQL for
  * foreign JDBC databases.
  */
-package org.eigenbase.rel.jdbc;
+package org.apache.calcite.rel.jdbc;
 
 // End package-info.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/logical/LogicalAggregate.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/logical/LogicalAggregate.java b/core/src/main/java/org/apache/calcite/rel/logical/LogicalAggregate.java
index 0e9c959..198f242 100644
--- a/core/src/main/java/org/apache/calcite/rel/logical/LogicalAggregate.java
+++ b/core/src/main/java/org/apache/calcite/rel/logical/LogicalAggregate.java
@@ -14,36 +14,44 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel;
+package org.apache.calcite.rel.logical;
 
-import java.util.*;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelInput;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelShuttle;
+import org.apache.calcite.rel.core.Aggregate;
+import org.apache.calcite.rel.core.AggregateCall;
 
-import org.eigenbase.relopt.*;
+import java.util.BitSet;
+import java.util.List;
 
 /**
- * <code>AggregateRel</code> is a relational operator which eliminates
+ * <code>LogicalAggregate</code> is a relational operator which eliminates
  * duplicates and computes totals.
  *
  * <p>Rules:
  *
  * <ul>
- * <li>{@link org.eigenbase.rel.rules.PullConstantsThroughAggregatesRule}
- * <li>{@link org.eigenbase.rel.rules.RemoveDistinctAggregateRule}
- * <li>{@link org.eigenbase.rel.rules.ReduceAggregatesRule}.
+ * <li>{@link org.apache.calcite.rel.rules.AggregateProjectPullUpConstantsRule}
+ * <li>{@link org.apache.calcite.rel.rules.AggregateExpandDistinctAggregatesRule}
+ * <li>{@link org.apache.calcite.rel.rules.AggregateReduceFunctionsRule}.
  * </ul>
  */
-public final class AggregateRel extends AggregateRelBase {
+public final class LogicalAggregate extends Aggregate {
   //~ Constructors -----------------------------------------------------------
 
   /**
-   * Creates an AggregateRel.
+   * Creates a LogicalAggregate.
    *
    * @param cluster  Cluster that this relational expression belongs to
    * @param child    input relational expression
    * @param groupSet Bit set of grouping fields
    * @param aggCalls Array of aggregates to compute, not null
    */
-  public AggregateRel(
+  public LogicalAggregate(
       RelOptCluster cluster,
       RelNode child,
       BitSet groupSet,
@@ -57,18 +65,18 @@ public final class AggregateRel extends AggregateRelBase {
   }
 
   /**
-   * Creates an AggregateRel by parsing serialized output.
+   * Creates a LogicalAggregate by parsing serialized output.
    */
-  public AggregateRel(RelInput input) {
+  public LogicalAggregate(RelInput input) {
     super(input);
   }
 
   //~ Methods ----------------------------------------------------------------
 
-  @Override public AggregateRel copy(RelTraitSet traitSet, RelNode input,
+  @Override public LogicalAggregate copy(RelTraitSet traitSet, RelNode input,
       BitSet groupSet, List<AggregateCall> aggCalls) {
     assert traitSet.containsIfApplicable(Convention.NONE);
-    return new AggregateRel(getCluster(), input, groupSet, aggCalls);
+    return new LogicalAggregate(getCluster(), input, groupSet, aggCalls);
   }
 
   @Override public RelNode accept(RelShuttle shuttle) {
@@ -76,4 +84,4 @@ public final class AggregateRel extends AggregateRelBase {
   }
 }
 
-// End AggregateRel.java
+// End LogicalAggregate.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/logical/LogicalCalc.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/logical/LogicalCalc.java b/core/src/main/java/org/apache/calcite/rel/logical/LogicalCalc.java
index ba0b650..3d74fab 100644
--- a/core/src/main/java/org/apache/calcite/rel/logical/LogicalCalc.java
+++ b/core/src/main/java/org/apache/calcite/rel/logical/LogicalCalc.java
@@ -14,41 +14,52 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel;
+package org.apache.calcite.rel.logical;
 
-import java.util.*;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Calc;
+import org.apache.calcite.rel.rules.FilterToCalcRule;
+import org.apache.calcite.rel.rules.ProjectToCalcRule;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
 
-import org.eigenbase.rel.rules.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.rex.*;
+import java.util.List;
+import java.util.Set;
 
 /**
  * A relational expression which computes project expressions and also filters.
  *
  * <p>This relational expression combines the functionality of
- * {@link ProjectRel} and {@link FilterRel}. It should be created in the later
- * stages of optimization, by merging consecutive {@link ProjectRel} and
- * {@link FilterRel} nodes together.
+ * {@link LogicalProject} and {@link LogicalFilter}.
+ * It should be created in the later
+ * stages of optimization, by merging consecutive {@link LogicalProject} and
+ * {@link LogicalFilter} nodes together.
  *
- * <p>The following rules relate to <code>CalcRel</code>:</p>
+ * <p>The following rules relate to <code>LogicalCalc</code>:</p>
  *
  * <ul>
- * <li>{@link FilterToCalcRule} creates this from a {@link FilterRel}</li>
- * <li>{@link ProjectToCalcRule} creates this from a {@link FilterRel}</li>
- * <li>{@link MergeFilterOntoCalcRule} merges this with a {@link FilterRel}</li>
- * <li>{@link MergeProjectOntoCalcRule} merges this with a
- *     {@link ProjectRel}</li>
- * <li>{@link MergeCalcRule} merges two CalcRels</li>
+ * <li>{@link FilterToCalcRule} creates this from a {@link LogicalFilter}
+ * <li>{@link ProjectToCalcRule} creates this from a {@link LogicalFilter}
+ * <li>{@link org.apache.calcite.rel.rules.FilterCalcMergeRule}
+ *     merges this with a {@link LogicalFilter}
+ * <li>{@link org.apache.calcite.rel.rules.ProjectCalcMergeRule}
+ *     merges this with a {@link LogicalProject}
+ * <li>{@link org.apache.calcite.rel.rules.CalcMergeRule}
+ *     merges two {@code LogicalCalc}s
  * </ul>
  */
-public final class CalcRel extends CalcRelBase {
+public final class LogicalCalc extends Calc {
   //~ Static fields/initializers ---------------------------------------------
 
   //~ Constructors -----------------------------------------------------------
 
-  /** Creates a CalcRel. */
-  public CalcRel(
+  /** Creates a LogicalCalc. */
+  public LogicalCalc(
       RelOptCluster cluster,
       RelTraitSet traits,
       RelNode child,
@@ -60,9 +71,9 @@ public final class CalcRel extends CalcRelBase {
 
   //~ Methods ----------------------------------------------------------------
 
-  @Override public CalcRelBase copy(RelTraitSet traitSet, RelNode child,
+  @Override public LogicalCalc copy(RelTraitSet traitSet, RelNode child,
       RexProgram program, List<RelCollation> collationList) {
-    return new CalcRel(getCluster(), traitSet, child,
+    return new LogicalCalc(getCluster(), traitSet, child,
         program.getOutputRowType(), program, collationList);
   }
 
@@ -76,4 +87,4 @@ public final class CalcRel extends CalcRelBase {
   }
 }
 
-// End CalcRel.java
+// End LogicalCalc.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/logical/LogicalFilter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/logical/LogicalFilter.java b/core/src/main/java/org/apache/calcite/rel/logical/LogicalFilter.java
index 2252e71..6766a78 100644
--- a/core/src/main/java/org/apache/calcite/rel/logical/LogicalFilter.java
+++ b/core/src/main/java/org/apache/calcite/rel/logical/LogicalFilter.java
@@ -14,32 +14,33 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel;
+package org.apache.calcite.rel.logical;
 
-import org.eigenbase.relopt.*;
-import org.eigenbase.rex.*;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelInput;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelShuttle;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rex.RexNode;
 
 /**
- * Relational expression that iterates over its input
- * and returns elements for which <code>condition</code> evaluates to
- * <code>true</code>.
- *
- * <p>If the condition allows nulls, then a null value is treated the same as
- * false.</p>
+ * Sub-class of {@link org.apache.calcite.rel.core.Filter}
+ * not targeted at any particular engine or calling convention.
  */
-public final class FilterRel extends FilterRelBase {
+public final class LogicalFilter extends Filter {
   //~ Constructors -----------------------------------------------------------
 
   /**
-   * Creates a filter.
+   * Creates a LogicalFilter.
    *
-   * @param cluster   {@link RelOptCluster}  this relational expression belongs
-   *                  to
-   * @param child     input relational expression
-   * @param condition boolean expression which determines whether a row is
+   * @param cluster   Cluster that this relational expression belongs to
+   * @param child     Input relational expression
+   * @param condition Boolean expression which determines whether a row is
    *                  allowed to pass
    */
-  public FilterRel(
+  public LogicalFilter(
       RelOptCluster cluster,
       RelNode child,
       RexNode condition) {
@@ -51,24 +52,23 @@ public final class FilterRel extends FilterRelBase {
   }
 
   /**
-   * Creates a FilterRel by parsing serialized output.
+   * Creates a LogicalFilter by parsing serialized output.
    */
-  public FilterRel(RelInput input) {
+  public LogicalFilter(RelInput input) {
     super(input);
   }
 
   //~ Methods ----------------------------------------------------------------
 
-  public FilterRel copy(RelTraitSet traitSet, RelNode input,
+  public LogicalFilter copy(RelTraitSet traitSet, RelNode input,
       RexNode condition) {
     assert traitSet.containsIfApplicable(Convention.NONE);
-    return new FilterRel(getCluster(), input, condition);
+    return new LogicalFilter(getCluster(), input, condition);
   }
 
-  @Override
-  public RelNode accept(RelShuttle shuttle) {
+  @Override public RelNode accept(RelShuttle shuttle) {
     return shuttle.visit(this);
   }
 }
 
-// End FilterRel.java
+// End LogicalFilter.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/logical/LogicalIntersect.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/logical/LogicalIntersect.java b/core/src/main/java/org/apache/calcite/rel/logical/LogicalIntersect.java
index 8bc3f67..5147b73 100644
--- a/core/src/main/java/org/apache/calcite/rel/logical/LogicalIntersect.java
+++ b/core/src/main/java/org/apache/calcite/rel/logical/LogicalIntersect.java
@@ -14,21 +14,26 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel;
+package org.apache.calcite.rel.logical;
 
-import java.util.List;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelInput;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelShuttle;
+import org.apache.calcite.rel.core.Intersect;
 
-import org.eigenbase.relopt.*;
+import java.util.List;
 
 /**
- * <code>IntersectRel</code> returns the intersection of the rows of its inputs.
- * If "all" is true, then multiset intersection is performed; otherwise, set
- * intersection is performed (implying no duplicates in the results).
+ * Sub-class of {@link org.apache.calcite.rel.core.Intersect}
+ * not targeted at any particular engine or calling convention.
  */
-public final class IntersectRel extends IntersectRelBase {
+public final class LogicalIntersect extends Intersect {
   //~ Constructors -----------------------------------------------------------
 
-  public IntersectRel(
+  public LogicalIntersect(
       RelOptCluster cluster,
       List<RelNode> inputs,
       boolean all) {
@@ -39,29 +44,22 @@ public final class IntersectRel extends IntersectRelBase {
         all);
   }
 
-  /**
-   * Creates an IntersectRel by parsing serialized output.
-   */
-  public IntersectRel(RelInput input) {
+  /** Creates a LogicalIntersect by parsing serialized output. */
+  public LogicalIntersect(RelInput input) {
     super(input);
   }
 
   //~ Methods ----------------------------------------------------------------
 
-  @Override
-  public IntersectRel copy(
-      RelTraitSet traitSet, List<RelNode> inputs, boolean all) {
+  @Override public LogicalIntersect copy(RelTraitSet traitSet,
+      List<RelNode> inputs, boolean all) {
     assert traitSet.containsIfApplicable(Convention.NONE);
-    return new IntersectRel(
-        getCluster(),
-        inputs,
-        all);
+    return new LogicalIntersect(getCluster(), inputs, all);
   }
 
-  @Override
-  public RelNode accept(RelShuttle shuttle) {
+  @Override public RelNode accept(RelShuttle shuttle) {
     return shuttle.visit(this);
   }
 }
 
-// End IntersectRel.java
+// End LogicalIntersect.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/logical/LogicalJoin.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/logical/LogicalJoin.java b/core/src/main/java/org/apache/calcite/rel/logical/LogicalJoin.java
index 8d06304..1dcea6f 100644
--- a/core/src/main/java/org/apache/calcite/rel/logical/LogicalJoin.java
+++ b/core/src/main/java/org/apache/calcite/rel/logical/LogicalJoin.java
@@ -14,30 +14,43 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel;
-
-import java.util.*;
-
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.RelDataTypeField;
-import org.eigenbase.rex.*;
+package org.apache.calcite.rel.logical;
+
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelInput;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelShuttle;
+import org.apache.calcite.rel.RelWriter;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexNode;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
 
+import java.util.List;
+import java.util.Set;
+
 /**
- * A JoinRel represents two relational expressions joined according to some
- * condition.
+ * Sub-class of {@link org.apache.calcite.rel.core.Join}
+ * not targeted at any particular engine or calling convention.
  *
  * <p>Some rules:
  *
- * <ul> <li>{@link org.eigenbase.rel.rules.ExtractJoinFilterRule} converts an
- * {@link JoinRel inner join} to a {@link FilterRel filter} on top of a {@link
- * JoinRel cartesian inner join}.  <li>{@code
- * net.sf.farrago.fennel.rel.FennelCartesianJoinRule} implements a JoinRel as a
- * cartesian product.  </ul>
+ * <ul>
+ * <li>{@link org.apache.calcite.rel.rules.JoinExtractFilterRule} converts an
+ * {@link LogicalJoin inner join} to a {@link LogicalFilter filter} on top of a
+ * {@link LogicalJoin cartesian inner join}.
+ *
+ * <li>{@code net.sf.farrago.fennel.rel.FennelCartesianJoinRule}
+ * implements a LogicalJoin as a cartesian product.
+ *
+ * </ul>
  */
-public final class JoinRel extends JoinRelBase {
+public final class LogicalJoin extends Join {
   //~ Instance fields --------------------------------------------------------
 
   // NOTE jvs 14-Mar-2006:  Normally we don't use state like this
@@ -50,7 +63,7 @@ public final class JoinRel extends JoinRelBase {
   //~ Constructors -----------------------------------------------------------
 
   /**
-   * Creates a JoinRel.
+   * Creates a LogicalJoin.
    *
    * @param cluster          Cluster
    * @param left             Left input
@@ -59,9 +72,9 @@ public final class JoinRel extends JoinRelBase {
    * @param joinType         Join type
    * @param variablesStopped Set of names of variables which are set by the
    *                         LHS and used by the RHS and are not available to
-   *                         nodes above this JoinRel in the tree
+   *                         nodes above this LogicalJoin in the tree
    */
-  public JoinRel(
+  public LogicalJoin(
       RelOptCluster cluster,
       RelNode left,
       RelNode right,
@@ -80,7 +93,7 @@ public final class JoinRel extends JoinRelBase {
   }
 
   /**
-   * Creates a JoinRel, flagged with whether it has been translated to a
+   * Creates a LogicalJoin, flagged with whether it has been translated to a
    * semi-join.
    *
    * @param cluster          Cluster
@@ -90,7 +103,7 @@ public final class JoinRel extends JoinRelBase {
    * @param joinType         Join type
    * @param variablesStopped Set of names of variables which are set by the
    *                         LHS and used by the RHS and are not available to
-   *                         nodes above this JoinRel in the tree
+   *                         nodes above this LogicalJoin in the tree
    * @param semiJoinDone     Whether this join has been translated to a
    *                         semi-join
    * @param systemFieldList  List of system fields that will be prefixed to
@@ -98,7 +111,7 @@ public final class JoinRel extends JoinRelBase {
    *                         null
    * @see #isSemiJoinDone()
    */
-  public JoinRel(
+  public LogicalJoin(
       RelOptCluster cluster,
       RelNode left,
       RelNode right,
@@ -121,9 +134,9 @@ public final class JoinRel extends JoinRelBase {
   }
 
   /**
-   * Creates a JoinRel by parsing serialized output.
+   * Creates a LogicalJoin by parsing serialized output.
    */
-  public JoinRel(RelInput input) {
+  public LogicalJoin(RelInput input) {
     this(
         input.getCluster(), input.getInputs().get(0),
         input.getInputs().get(1), input.getExpression("condition"),
@@ -134,11 +147,10 @@ public final class JoinRel extends JoinRelBase {
 
   //~ Methods ----------------------------------------------------------------
 
-  @Override
-  public JoinRel copy(RelTraitSet traitSet, RexNode conditionExpr, RelNode left,
-      RelNode right, JoinRelType joinType, boolean semiJoinDone) {
+  @Override public LogicalJoin copy(RelTraitSet traitSet, RexNode conditionExpr,
+      RelNode left, RelNode right, JoinRelType joinType, boolean semiJoinDone) {
     assert traitSet.containsIfApplicable(Convention.NONE);
-    return new JoinRel(
+    return new LogicalJoin(
         getCluster(),
         left,
         right,
@@ -149,8 +161,7 @@ public final class JoinRel extends JoinRelBase {
         this.systemFieldList);
   }
 
-  @Override
-  public RelNode accept(RelShuttle shuttle) {
+  @Override public RelNode accept(RelShuttle shuttle) {
     return shuttle.visit(this);
   }
 
@@ -170,4 +181,4 @@ public final class JoinRel extends JoinRelBase {
   }
 }
 
-// End JoinRel.java
+// End LogicalJoin.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/logical/LogicalMinus.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/logical/LogicalMinus.java b/core/src/main/java/org/apache/calcite/rel/logical/LogicalMinus.java
index 83411a9..9db2a1b 100644
--- a/core/src/main/java/org/apache/calcite/rel/logical/LogicalMinus.java
+++ b/core/src/main/java/org/apache/calcite/rel/logical/LogicalMinus.java
@@ -14,22 +14,26 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel;
+package org.apache.calcite.rel.logical;
 
-import java.util.List;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelInput;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelShuttle;
+import org.apache.calcite.rel.core.Minus;
 
-import org.eigenbase.relopt.*;
+import java.util.List;
 
 /**
- * <code>MinusRel</code> returns the rows of its first input minus any matching
- * rows from its other inputs. If "all" is true, then multiset subtraction is
- * performed; otherwise, set subtraction is performed (implying no duplicates in
- * the results).
+ * Sub-class of {@link org.apache.calcite.rel.core.Minus}
+ * not targeted at any particular engine or calling convention.
  */
-public final class MinusRel extends MinusRelBase {
+public final class LogicalMinus extends Minus {
   //~ Constructors -----------------------------------------------------------
 
-  public MinusRel(
+  public LogicalMinus(
       RelOptCluster cluster,
       List<RelNode> inputs,
       boolean all) {
@@ -41,28 +45,26 @@ public final class MinusRel extends MinusRelBase {
   }
 
   /**
-   * Creates a MinusRel by parsing serialized output.
+   * Creates a LogicalMinus by parsing serialized output.
    */
-  public MinusRel(RelInput input) {
+  public LogicalMinus(RelInput input) {
     super(input);
   }
 
   //~ Methods ----------------------------------------------------------------
 
-  @Override
-  public MinusRel copy(
+  @Override public LogicalMinus copy(
       RelTraitSet traitSet, List<RelNode> inputs, boolean all) {
     assert traitSet.containsIfApplicable(Convention.NONE);
-    return new MinusRel(
+    return new LogicalMinus(
         getCluster(),
         inputs,
         all);
   }
 
-  @Override
-  public RelNode accept(RelShuttle shuttle) {
+  @Override public RelNode accept(RelShuttle shuttle) {
     return shuttle.visit(this);
   }
 }
 
-// End MinusRel.java
+// End LogicalMinus.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/logical/LogicalOneRow.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/logical/LogicalOneRow.java b/core/src/main/java/org/apache/calcite/rel/logical/LogicalOneRow.java
index 3e56391..4a44459 100644
--- a/core/src/main/java/org/apache/calcite/rel/logical/LogicalOneRow.java
+++ b/core/src/main/java/org/apache/calcite/rel/logical/LogicalOneRow.java
@@ -14,44 +14,47 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel;
+package org.apache.calcite.rel.logical;
 
-import java.util.List;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelInput;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.OneRow;
 
-import org.eigenbase.relopt.*;
+import java.util.List;
 
 /**
- * <code>OneRowRel</code> always returns one row, one column (containing the
- * value 0).
+ * Sub-class of {@link org.apache.calcite.rel.core.OneRow}
+ * not targeted at any particular engine or calling convention.
  */
-public final class OneRowRel extends OneRowRelBase {
+public final class LogicalOneRow extends OneRow {
   //~ Constructors -----------------------------------------------------------
 
   /**
-   * Creates a <code>OneRowRel</code>.
+   * Creates a <code>LogicalOneRow</code>.
    *
-   * @param cluster {@link RelOptCluster}  this relational expression belongs
-   *                to
+   * @param cluster   Cluster that this relational expression belongs to
    */
-  public OneRowRel(RelOptCluster cluster) {
+  public LogicalOneRow(RelOptCluster cluster) {
     super(
         cluster,
         cluster.traitSetOf(Convention.NONE));
   }
 
   /**
-   * Creates a OneRowRel by parsing serialized output.
+   * Creates a LogicalOneRow by parsing serialized output.
    */
-  public OneRowRel(RelInput input) {
+  public LogicalOneRow(RelInput input) {
     super(input.getCluster(), input.getTraitSet());
   }
 
-  @Override
-  public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+  @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
     assert traitSet.containsIfApplicable(Convention.NONE);
     assert inputs.isEmpty();
     return this;
   }
 }
 
-// End OneRowRel.java
+// End LogicalOneRow.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/logical/LogicalProject.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/logical/LogicalProject.java b/core/src/main/java/org/apache/calcite/rel/logical/LogicalProject.java
index faec719..8cc81a5 100644
--- a/core/src/main/java/org/apache/calcite/rel/logical/LogicalProject.java
+++ b/core/src/main/java/org/apache/calcite/rel/logical/LogicalProject.java
@@ -14,35 +14,40 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel;
+package org.apache.calcite.rel.logical;
 
-import java.util.*;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollationImpl;
+import org.apache.calcite.rel.RelInput;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelShuttle;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexUtil;
 
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.rex.*;
+import java.util.List;
 
 /**
- * <code>ProjectRel</code> is a relational expression which computes a set of
- * 'select expressions' from its input relational expression.
- *
- * <p>The result is usually 'boxed' as a record with one named field for each
- * column; if there is precisely one expression, the result may be 'unboxed',
- * and consist of the raw value type.</p>
+ * Sub-class of {@link org.apache.calcite.rel.core.Project} not
+ * targeted at any particular engine or calling convention.
  */
-public final class ProjectRel extends ProjectRelBase {
+public final class LogicalProject extends Project {
   //~ Constructors -----------------------------------------------------------
 
   /**
-   * Creates a ProjectRel with no sort keys.
+   * Creates a LogicalProject with no sort keys.
    *
    * @param cluster    Cluster this relational expression belongs to
    * @param child      input relational expression
    * @param exps       set of expressions for the input columns
    * @param fieldNames aliases of the expressions
-   * @param flags      values as in {@link ProjectRelBase.Flags}
+   * @param flags      Flags; values as in {@link Project.Flags},
+   *                   usually {@link Project.Flags#BOXED}
    */
-  public ProjectRel(
+  public LogicalProject(
       RelOptCluster cluster,
       RelNode child,
       List<RexNode> exps,
@@ -61,16 +66,17 @@ public final class ProjectRel extends ProjectRelBase {
   }
 
   /**
-   * Creates a ProjectRel.
+   * Creates a LogicalProject.
    *
    * @param cluster  Cluster this relational expression belongs to
    * @param traitSet traits of this rel
    * @param child    input relational expression
    * @param exps     List of expressions for the input columns
    * @param rowType  output row type
-   * @param flags    values as in {@link ProjectRelBase.Flags}
+   * @param flags      Flags; values as in {@link Project.Flags},
+   *                   usually {@link Project.Flags#BOXED}
    */
-  public ProjectRel(
+  public LogicalProject(
       RelOptCluster cluster,
       RelTraitSet traitSet,
       RelNode child,
@@ -82,23 +88,23 @@ public final class ProjectRel extends ProjectRelBase {
   }
 
   /**
-   * Creates a ProjectRel by parsing serialized output.
+   * Creates a LogicalProject by parsing serialized output.
    */
-  public ProjectRel(RelInput input) {
+  public LogicalProject(RelInput input) {
     super(input);
   }
 
   //~ Methods ----------------------------------------------------------------
 
-  @Override public ProjectRel copy(RelTraitSet traitSet, RelNode input,
+  @Override public LogicalProject copy(RelTraitSet traitSet, RelNode input,
       List<RexNode> exps, RelDataType rowType) {
-    return new ProjectRel(getCluster(), traitSet, input, exps, rowType, flags);
+    return new LogicalProject(getCluster(), traitSet, input, exps, rowType,
+        flags);
   }
 
-  @Override
-  public RelNode accept(RelShuttle shuttle) {
+  @Override public RelNode accept(RelShuttle shuttle) {
     return shuttle.visit(this);
   }
 }
 
-// End ProjectRel.java
+// End LogicalProject.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/logical/LogicalTableFunctionScan.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/logical/LogicalTableFunctionScan.java b/core/src/main/java/org/apache/calcite/rel/logical/LogicalTableFunctionScan.java
index d2ec6a8..de0d1fa 100644
--- a/core/src/main/java/org/apache/calcite/rel/logical/LogicalTableFunctionScan.java
+++ b/core/src/main/java/org/apache/calcite/rel/logical/LogicalTableFunctionScan.java
@@ -14,27 +14,33 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel;
+package org.apache.calcite.rel.logical;
+
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelInput;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.TableFunctionScan;
+import org.apache.calcite.rel.metadata.RelColumnMapping;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexNode;
 
 import java.lang.reflect.Type;
 import java.util.List;
 import java.util.Set;
 
-import org.eigenbase.rel.metadata.RelColumnMapping;
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.rex.*;
-
 /**
- * <code>TableFunctionRel</code> represents a call to a function which returns a
- * result set. Currently, it can only appear as a leaf in a query tree, but
- * eventually we will extend it to take relational inputs.
+ * Sub-class of {@link org.apache.calcite.rel.core.TableFunctionScan}
+ * not targeted at any particular engine or calling convention.
  */
-public class TableFunctionRel extends TableFunctionRelBase {
+public class LogicalTableFunctionScan extends TableFunctionScan {
   //~ Constructors -----------------------------------------------------------
 
   /**
-   * Creates a <code>TableFunctionRel</code>.
+   * Creates a <code>LogicalTableFunctionScan</code>.
    *
    * @param cluster        Cluster that this relational expression belongs to
    * @param inputs         0 or more relational inputs
@@ -44,7 +50,7 @@ public class TableFunctionRel extends TableFunctionRelBase {
    * @param rowType        row type produced by function
    * @param columnMappings column mappings associated with this function
    */
-  public TableFunctionRel(
+  public LogicalTableFunctionScan(
       RelOptCluster cluster,
       List<RelNode> inputs,
       RexNode rexCall,
@@ -61,18 +67,18 @@ public class TableFunctionRel extends TableFunctionRelBase {
   }
 
   /**
-   * Creates a TableFunctionRel by parsing serialized output.
+   * Creates a LogicalTableFunctionScan by parsing serialized output.
    */
-  public TableFunctionRel(RelInput input) {
+  public LogicalTableFunctionScan(RelInput input) {
     super(input);
   }
 
   //~ Methods ----------------------------------------------------------------
 
-  @Override
-  public TableFunctionRel copy(RelTraitSet traitSet, List<RelNode> inputs) {
+  @Override public LogicalTableFunctionScan copy(RelTraitSet traitSet,
+      List<RelNode> inputs) {
     assert traitSet.containsIfApplicable(Convention.NONE);
-    return new TableFunctionRel(
+    return new LogicalTableFunctionScan(
         getCluster(),
         inputs,
         getCall(),
@@ -87,4 +93,4 @@ public class TableFunctionRel extends TableFunctionRelBase {
   }
 }
 
-// End TableFunctionRel.java
+// End LogicalTableFunctionScan.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/logical/LogicalTableModify.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/logical/LogicalTableModify.java b/core/src/main/java/org/apache/calcite/rel/logical/LogicalTableModify.java
index b2b1246..c1835ff 100644
--- a/core/src/main/java/org/apache/calcite/rel/logical/LogicalTableModify.java
+++ b/core/src/main/java/org/apache/calcite/rel/logical/LogicalTableModify.java
@@ -14,24 +14,26 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel;
+package org.apache.calcite.rel.logical;
 
-import java.util.*;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.prepare.Prepare;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.TableModify;
 
-import org.eigenbase.relopt.*;
-
-import net.hydromatic.optiq.prepare.Prepare;
+import java.util.List;
 
 /**
- * TableModificationRel is like TableAccessRel, but represents a request to
- * modify a table rather than read from it. It takes one child which produces
- * the modified rows. (For INSERT, the new values; for DELETE, the old values;
- * for UPDATE, all old values plus updated new values.)
+ * Sub-class of {@link org.apache.calcite.rel.core.TableModify}
+ * not targeted at any particular engine or calling convention.
  */
-public final class TableModificationRel extends TableModificationRelBase {
+public final class LogicalTableModify extends TableModify {
   //~ Constructors -----------------------------------------------------------
 
-  public TableModificationRel(
+  public LogicalTableModify(
       RelOptCluster cluster,
       RelOptTable table,
       Prepare.CatalogReader schema,
@@ -52,10 +54,10 @@ public final class TableModificationRel extends TableModificationRelBase {
 
   //~ Methods ----------------------------------------------------------------
 
-  @Override
-  public TableModificationRel copy(RelTraitSet traitSet, List<RelNode> inputs) {
+  @Override public LogicalTableModify copy(RelTraitSet traitSet,
+      List<RelNode> inputs) {
     assert traitSet.containsIfApplicable(Convention.NONE);
-    return new TableModificationRel(
+    return new LogicalTableModify(
         getCluster(),
         table,
         catalogReader,
@@ -66,4 +68,4 @@ public final class TableModificationRel extends TableModificationRelBase {
   }
 }
 
-// End TableModificationRel.java
+// End LogicalTableModify.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/logical/LogicalTableScan.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/logical/LogicalTableScan.java b/core/src/main/java/org/apache/calcite/rel/logical/LogicalTableScan.java
index 5341fc2..5340fd0 100644
--- a/core/src/main/java/org/apache/calcite/rel/logical/LogicalTableScan.java
+++ b/core/src/main/java/org/apache/calcite/rel/logical/LogicalTableScan.java
@@ -14,14 +14,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel;
+package org.apache.calcite.rel.logical;
 
-import java.util.List;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelInput;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.TableScan;
 
-import org.eigenbase.relopt.*;
+import java.util.List;
 
 /**
- * A <code>TableAccessRel</code> reads all the rows from a {@link RelOptTable}.
+ * A <code>LogicalTableScan</code> reads all the rows from a
+ * {@link RelOptTable}.
  *
  * <p>If the table is a <code>net.sf.saffron.ext.JdbcTable</code>, then this is
  * literally possible. But for other kinds of tables, there may be many ways to
@@ -45,16 +52,16 @@ import org.eigenbase.relopt.*;
  * <p>can. It is the optimizer's responsibility to find these ways, by applying
  * transformation rules.</p>
  */
-public final class TableAccessRel extends TableAccessRelBase {
+public final class LogicalTableScan extends TableScan {
   //~ Constructors -----------------------------------------------------------
 
   /**
-   * Creates a TableAccessRel.
+   * Creates a LogicalTableScan.
    *
    * @param cluster Cluster
    * @param table   Table
    */
-  public TableAccessRel(
+  public LogicalTableScan(
       RelOptCluster cluster,
       RelOptTable table) {
     super(
@@ -64,18 +71,17 @@ public final class TableAccessRel extends TableAccessRelBase {
   }
 
   /**
-   * Creates a TableAccessRel by parsing serialized output.
+   * Creates a LogicalTableScan by parsing serialized output.
    */
-  public TableAccessRel(RelInput input) {
+  public LogicalTableScan(RelInput input) {
     super(input);
   }
 
-  @Override
-  public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+  @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
     assert traitSet.containsIfApplicable(Convention.NONE);
     assert inputs.isEmpty();
     return this;
   }
 }
 
-// End TableAccessRel.java
+// End LogicalTableScan.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/logical/LogicalUnion.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/logical/LogicalUnion.java b/core/src/main/java/org/apache/calcite/rel/logical/LogicalUnion.java
index 4e24846..49dde4d 100644
--- a/core/src/main/java/org/apache/calcite/rel/logical/LogicalUnion.java
+++ b/core/src/main/java/org/apache/calcite/rel/logical/LogicalUnion.java
@@ -14,20 +14,26 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel;
+package org.apache.calcite.rel.logical;
 
-import java.util.List;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelInput;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelShuttle;
+import org.apache.calcite.rel.core.Union;
 
-import org.eigenbase.relopt.*;
+import java.util.List;
 
 /**
- * <code>UnionRel</code> returns the union of the rows of its inputs, optionally
- * eliminating duplicates.
+ * Sub-class of {@link org.apache.calcite.rel.core.Union}
+ * not targeted at any particular engine or calling convention.
  */
-public final class UnionRel extends UnionRelBase {
+public final class LogicalUnion extends Union {
   //~ Constructors -----------------------------------------------------------
 
-  public UnionRel(
+  public LogicalUnion(
       RelOptCluster cluster,
       List<RelNode> inputs,
       boolean all) {
@@ -39,27 +45,26 @@ public final class UnionRel extends UnionRelBase {
   }
 
   /**
-   * Creates a UnionRel by parsing serialized output.
+   * Creates a LogicalUnion by parsing serialized output.
    */
-  public UnionRel(RelInput input) {
+  public LogicalUnion(RelInput input) {
     super(input);
   }
 
   //~ Methods ----------------------------------------------------------------
 
-  public UnionRel copy(
+  public LogicalUnion copy(
       RelTraitSet traitSet, List<RelNode> inputs, boolean all) {
     assert traitSet.containsIfApplicable(Convention.NONE);
-    return new UnionRel(
+    return new LogicalUnion(
         getCluster(),
         inputs,
         all);
   }
 
-  @Override
-  public RelNode accept(RelShuttle shuttle) {
+  @Override public RelNode accept(RelShuttle shuttle) {
     return shuttle.visit(this);
   }
 }
 
-// End UnionRel.java
+// End LogicalUnion.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/logical/LogicalValues.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/logical/LogicalValues.java b/core/src/main/java/org/apache/calcite/rel/logical/LogicalValues.java
index 5b28d8e..fc3beed 100644
--- a/core/src/main/java/org/apache/calcite/rel/logical/LogicalValues.java
+++ b/core/src/main/java/org/apache/calcite/rel/logical/LogicalValues.java
@@ -14,63 +14,66 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel;
+package org.apache.calcite.rel.logical;
 
-import java.util.*;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelInput;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelShuttle;
+import org.apache.calcite.rel.core.Values;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexLiteral;
 
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.rex.*;
+import java.util.List;
 
 /**
- * <code>ValuesRel</code> represents a sequence of zero or more literal row
- * values.
+ * Sub-class of {@link org.apache.calcite.rel.core.Values}
+ * not targeted at any particular engine or calling convention.
  */
-public class ValuesRel extends ValuesRelBase {
+public class LogicalValues extends Values {
   //~ Constructors -----------------------------------------------------------
 
   /**
-   * Creates a new ValuesRel. Note that tuples passed in become owned by this
+   * Creates a new LogicalValues.
+   *
+   * <p>Note that tuples passed in become owned by this
    * rel (without a deep copy), so caller must not modify them after this
    * call, otherwise bad things will happen.
    *
-   * @param cluster .
-   * @param rowType row type for tuples produced by this rel
+   * @param cluster Cluster that this relational expression belongs to
+   * @param rowType Row type for tuples produced by this rel
    * @param tuples  2-dimensional array of tuple values to be produced; outer
    *                list contains tuples; each inner list is one tuple; all
    *                tuples must be of same length, conforming to rowType
    */
-  public ValuesRel(
+  public LogicalValues(
       RelOptCluster cluster,
       RelDataType rowType,
       List<List<RexLiteral>> tuples) {
-    super(
-        cluster,
-        rowType,
-        tuples,
-        cluster.traitSetOf(Convention.NONE));
+    super(cluster, rowType, tuples, cluster.traitSetOf(Convention.NONE));
   }
 
   /**
-   * Creates a ValuesRel by parsing serialized output.
+   * Creates a LogicalValues by parsing serialized output.
    */
-  public ValuesRel(RelInput input) {
+  public LogicalValues(RelInput input) {
     super(input);
   }
 
-  public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+  @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
     assert traitSet.containsIfApplicable(Convention.NONE);
     assert inputs.isEmpty();
-    return new ValuesRel(
+    return new LogicalValues(
         getCluster(),
         rowType,
         tuples);
   }
 
-  @Override
-  public RelNode accept(RelShuttle shuttle) {
+  @Override public RelNode accept(RelShuttle shuttle) {
     return shuttle.visit(this);
   }
 }
 
-// End ValuesRel.java
+// End LogicalValues.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 d278a38..e75d340 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
@@ -14,61 +14,70 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel;
-
-import java.util.*;
-
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.rex.*;
-import org.eigenbase.util.Pair;
-import org.eigenbase.util.Util;
-
-import net.hydromatic.linq4j.Ord;
-
-import net.hydromatic.optiq.util.BitSets;
-
+package org.apache.calcite.rel.logical;
+
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Window;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexFieldCollation;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexLocalRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexOver;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexShuttle;
+import org.apache.calcite.rex.RexWindow;
+import org.apache.calcite.rex.RexWindowBound;
+import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.Pair;
+
+import com.google.common.base.Objects;
 import com.google.common.base.Predicate;
-import com.google.common.collect.*;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.LinkedListMultimap;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Multimap;
+
+import java.util.AbstractList;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
 
 /**
- * A relational expression representing a set of window aggregates.
- *
- * <p>A window rel can handle several window aggregate functions, over several
- * partitions, with pre- and post-expressions, and an optional post-filter.
- * Each of the partitions is defined by a partition key (zero or more columns)
- * and a range (logical or physical). The partitions expect the data to be
- * sorted correctly on input to the relational expression.
- *
- * <p>Each {@link org.eigenbase.rel.WindowRelBase.Window} has a set of
- * {@link org.eigenbase.rex.RexOver} objects.
- *
- * <p>Created by {@link org.eigenbase.rel.rules.WindowedAggSplitterRule}.
  */
-public final class WindowRel extends WindowRelBase {
+public final class LogicalWindow extends Window {
   /**
-   * Creates a WindowRel.
+   * Creates a LogicalWindow.
    *
    * @param cluster Cluster
    * @param child   Input relational expression
    * @param constants List of constants that are additional inputs
    * @param rowType Output row type
-   * @param windows Windows
+   * @param groups Windows
    */
-  public WindowRel(
+  public LogicalWindow(
       RelOptCluster cluster, RelTraitSet traits, RelNode child,
-      List<RexLiteral> constants, RelDataType rowType, List<Window> windows) {
-    super(cluster, traits, child, constants, rowType, windows);
+      List<RexLiteral> constants, RelDataType rowType, List<Group> groups) {
+    super(cluster, traits, child, constants, rowType, groups);
   }
 
-  @Override
-  public WindowRel copy(RelTraitSet traitSet, List<RelNode> inputs) {
-    return new WindowRel(
-        getCluster(), traitSet, sole(inputs), constants, rowType, windows);
+  @Override public LogicalWindow copy(RelTraitSet traitSet,
+      List<RelNode> inputs) {
+    return new LogicalWindow(getCluster(), traitSet, sole(inputs), constants,
+      rowType, groups);
   }
 
   /**
-   * Creates a WindowRel.
+   * Creates a LogicalWindow.
    */
   public static RelNode create(
       RelOptCluster cluster,
@@ -76,7 +85,7 @@ public final class WindowRel extends WindowRelBase {
       RelNode child,
       final RexProgram program,
       RelDataType outRowType) {
-    // Build a list of distinct windows, partitions and aggregate
+    // Build a list of distinct groups, partitions and aggregate
     // functions.
     final Multimap<WindowKey, RexOver> windowMap =
         LinkedListMultimap.create();
@@ -90,8 +99,7 @@ public final class WindowRel extends WindowRelBase {
     // Identify constants in the expression tree and replace them with
     // references to newly generated constant pool.
     RexShuttle replaceConstants = new RexShuttle() {
-      @Override
-      public RexNode visitLiteral(RexLiteral literal) {
+      @Override public RexNode visitLiteral(RexLiteral literal) {
         RexInputRef ref = constantPool.get(literal);
         if (ref != null) {
           return ref;
@@ -104,7 +112,7 @@ public final class WindowRel extends WindowRelBase {
       }
     };
 
-    // Build a list of windows, partitions, and aggregate functions. Each
+    // Build a list of groups, partitions, and aggregate functions. Each
     // aggregate function will add its arguments as outputs of the input
     // program.
     for (RexNode agg : program.getExprList()) {
@@ -115,9 +123,9 @@ public final class WindowRel extends WindowRelBase {
       }
     }
 
-    final Map<RexOver, WindowRelBase.RexWinAggCall> aggMap =
-        new HashMap<RexOver, WindowRelBase.RexWinAggCall>();
-    List<Window> windowList = new ArrayList<Window>();
+    final Map<RexOver, Window.RexWinAggCall> aggMap =
+        new HashMap<RexOver, Window.RexWinAggCall>();
+    List<Group> groups = new ArrayList<Group>();
     for (Map.Entry<WindowKey, Collection<RexOver>> entry
         : windowMap.asMap().entrySet()) {
       final WindowKey windowKey = entry.getKey();
@@ -134,13 +142,12 @@ public final class WindowRel extends WindowRelBase {
         aggMap.put(over, aggCall);
       }
       RexShuttle toInputRefs = new RexShuttle() {
-        @Override
-        public RexNode visitLocalRef(RexLocalRef localRef) {
+        @Override public RexNode visitLocalRef(RexLocalRef localRef) {
           return new RexInputRef(localRef.getIndex(), localRef.getType());
         }
       };
-      windowList.add(
-          new Window(
+      groups.add(
+          new Group(
               windowKey.groupSet,
               windowKey.isRows,
               windowKey.lowerBound.accept(toInputRefs),
@@ -152,8 +159,8 @@ public final class WindowRel extends WindowRelBase {
     // Figure out the type of the inputs to the output program.
     // They are: the inputs to this rel, followed by the outputs of
     // each window.
-    final List<WindowRelBase.RexWinAggCall> flattenedAggCallList =
-        new ArrayList<WindowRelBase.RexWinAggCall>();
+    final List<Window.RexWinAggCall> flattenedAggCallList =
+        new ArrayList<Window.RexWinAggCall>();
     List<Map.Entry<String, RelDataType>> fieldList =
         new ArrayList<Map.Entry<String, RelDataType>>(
             child.getRowType().getFieldList());
@@ -169,7 +176,7 @@ public final class WindowRel extends WindowRelBase {
       }
     }
 
-    for (Ord<Window> window : Ord.zip(windowList)) {
+    for (Ord<Group> window : Ord.zip(groups)) {
       for (Ord<RexWinAggCall> over : Ord.zip(window.e.aggCalls)) {
         // Add the k-th over expression of
         // the i-th window to the output of the program.
@@ -190,7 +197,7 @@ public final class WindowRel extends WindowRelBase {
         new RexShuttle() {
           public RexNode visitOver(RexOver over) {
             // Look up the aggCall which this expr was translated to.
-            final WindowRelBase.RexWinAggCall aggCall =
+            final Window.RexWinAggCall aggCall =
                 aggMap.get(over);
             assert aggCall != null;
             assert RelOptUtil.eq(
@@ -201,7 +208,7 @@ public final class WindowRel extends WindowRelBase {
                 true);
 
             // Find the index of the aggCall among all partitions of all
-            // windows.
+            // groups.
             final int aggCallIndex =
                 flattenedAggCallList.indexOf(aggCall);
             assert aggCallIndex >= 0;
@@ -230,14 +237,14 @@ public final class WindowRel extends WindowRelBase {
                 localRef.getType());
           }
         };
-    // TODO: The order that the "over" calls occur in the windows and
+    // TODO: The order that the "over" calls occur in the groups and
     // partitions may not match the order in which they occurred in the
     // original expression. We should add a project to permute them.
 
-    WindowRel window =
-        new WindowRel(
+    LogicalWindow window =
+        new LogicalWindow(
             cluster, traitSet, child, constants, intermediateRowType,
-            windowList);
+            groups);
 
     return RelOptUtil.createProject(
         window,
@@ -264,7 +271,7 @@ public final class WindowRel extends WindowRelBase {
     };
   }
 
-  /** Window specification. All windowed aggregates over the same window
+  /** Group specification. All windowed aggregates over the same window
    * (regardless of how it is specified, in terms of a named window or specified
    * attribute by attribute) will end up with the same window key. */
   private static class WindowKey {
@@ -287,20 +294,21 @@ public final class WindowRel extends WindowRelBase {
       this.upperBound = upperBound;
     }
 
-    @Override
-    public int hashCode() {
-      return Util.hashV(
-          groupSet, orderKeys, isRows, lowerBound, upperBound);
+    @Override public int hashCode() {
+      return com.google.common.base.Objects.hashCode(groupSet,
+          orderKeys,
+          isRows,
+          lowerBound,
+          upperBound);
     }
 
-    @Override
-    public boolean equals(Object obj) {
+    @Override public boolean equals(Object obj) {
       return obj == this
           || obj instanceof WindowKey
           && groupSet.equals(((WindowKey) obj).groupSet)
           && orderKeys.equals(((WindowKey) obj).orderKeys)
-          && Util.equal(lowerBound, ((WindowKey) obj).lowerBound)
-          && Util.equal(upperBound, ((WindowKey) obj).upperBound)
+          && Objects.equal(lowerBound, ((WindowKey) obj).lowerBound)
+          && Objects.equal(upperBound, ((WindowKey) obj).upperBound)
           && isRows == ((WindowKey) obj).isRows;
     }
   }
@@ -338,4 +346,4 @@ public final class WindowRel extends WindowRelBase {
   }
 }
 
-// End WindowRel.java
+// End LogicalWindow.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/logical/package-info.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/logical/package-info.java b/core/src/main/java/org/apache/calcite/rel/logical/package-info.java
new file mode 100644
index 0000000..3317ac1
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/rel/logical/package-info.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Defines logical relational expressions.
+ *
+ * <h2>Related packages and classes</h2>
+ * <ul>
+ *
+ * <li>Package <code>
+ * <a href="../logical/package-summary.html">org.apache.calcite.rel.core</a></code>
+ * contains core relational expressions
+ *
+ * <li>Package <code>
+ * <a href="../package-summary.html">org.apache.calcite.rex</a></code>
+ * defines the relational expression API
+ *
+ * </ul>
+ */
+package org.apache.calcite.rel.logical;
+
+// End package-info.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/metadata/BuiltInMetadata.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/BuiltInMetadata.java b/core/src/main/java/org/apache/calcite/rel/metadata/BuiltInMetadata.java
index 7bb40ec..4d890b6 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/BuiltInMetadata.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/BuiltInMetadata.java
@@ -14,15 +14,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.metadata;
+package org.apache.calcite.rel.metadata;
 
-import java.util.*;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPredicateList;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.SqlExplainLevel;
 
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.RelOptCost;
-import org.eigenbase.relopt.RelOptPredicateList;
-import org.eigenbase.rex.RexNode;
-import org.eigenbase.sql.SqlExplainLevel;
+import java.util.BitSet;
+import java.util.Set;
 
 /**
  * Contains the interfaces for several common forms of metadata.
@@ -137,7 +138,7 @@ public abstract class BuiltInMetadata {
      * Estimates the distinct row count in the original source for the given
      * {@code groupKey}, ignoring any filtering being applied by the expression.
      * Typically, "original source" means base table, but for derived columns,
-     * the estimate may come from a non-leaf rel such as a ProjectRel.
+     * the estimate may come from a non-leaf rel such as a LogicalProject.
      *
      * @param groupKey column mask representing the subset of columns for which
      *                 the row count will be determined
@@ -152,8 +153,8 @@ public abstract class BuiltInMetadata {
     /**
      * For a given output column of an expression, determines all columns of
      * underlying tables which contribute to result values. An output column may
-     * have more than one origin due to expressions such as UnionRel and
-     * ProjectRel. The optimizer may use this information for catalog access
+     * have more than one origin due to expressions such as Union and
+     * LogicalProject. The optimizer may use this information for catalog access
      * (e.g. index availability).
      *
      * @param outputColumn 0-based ordinal for output column of interest

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 fe825c0..a45501b 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
@@ -14,17 +14,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.metadata;
+package org.apache.calcite.rel.metadata;
 
-import java.lang.reflect.*;
-import java.util.*;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.rel.RelNode;
 
 import com.google.common.base.Function;
 import com.google.common.collect.ImmutableList;
 
+import java.lang.reflect.InvocationHandler;
+import java.lang.reflect.Method;
+import java.lang.reflect.Proxy;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
 /**
  * Implementation of the {@link RelMetadataProvider}
  * interface that caches results from an underlying provider.
@@ -39,8 +43,7 @@ public class CachingRelMetadataProvider implements RelMetadataProvider {
   private final RelOptPlanner planner;
 
   private static final Object NULL_SENTINEL = new Object() {
-    @Override
-    public String toString() {
+    @Override public String toString() {
       return "{null}";
     }
   };

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 3cef5b3..445cce4 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
@@ -14,21 +14,23 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.metadata;
+package org.apache.calcite.rel.metadata;
 
-import java.lang.reflect.*;
-import java.util.*;
-
-import org.eigenbase.rel.*;
+import org.apache.calcite.rel.RelNode;
 
 import com.google.common.base.Function;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 
+import java.lang.reflect.InvocationHandler;
+import java.lang.reflect.Method;
+import java.lang.reflect.Proxy;
+import java.util.List;
+
 /**
  * Implementation of the {@link RelMetadataProvider}
  * interface via the
- * {@link org.eigenbase.util.Glossary#CHAIN_OF_RESPONSIBILITY_PATTERN}.
+ * {@link org.apache.calcite.util.Glossary#CHAIN_OF_RESPONSIBILITY_PATTERN}.
  *
  * <p>When a consumer calls the {@link #apply} method to ask for a provider
  * for a particular type of {@link RelNode} and {@link Metadata}, scans the list

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/metadata/DefaultRelMetadataProvider.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/DefaultRelMetadataProvider.java b/core/src/main/java/org/apache/calcite/rel/metadata/DefaultRelMetadataProvider.java
index 5958476..33ad92f 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/DefaultRelMetadataProvider.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/DefaultRelMetadataProvider.java
@@ -14,15 +14,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.metadata;
+package org.apache.calcite.rel.metadata;
 
 import com.google.common.collect.ImmutableList;
 
 /**
- * DefaultRelMetadataProvider supplies a default implementation of the {@link
- * RelMetadataProvider} interface. It provides generic formulas and derivation
- * rules for the standard logical algebra; coverage corresponds to the methods
- * declared in {@link RelMetadataQuery}.
+ * DefaultRelMetadataProvider supplies a default implementation of the
+ * {@link RelMetadataProvider} interface. It provides generic formulas and
+ * derivation rules for the standard logical algebra; coverage corresponds to
+ * the methods declared in {@link RelMetadataQuery}.
  */
 public class DefaultRelMetadataProvider extends ChainedRelMetadataProvider {
   //~ Constructors -----------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/metadata/Metadata.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/Metadata.java b/core/src/main/java/org/apache/calcite/rel/metadata/Metadata.java
index be97533..34e456f 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/Metadata.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/Metadata.java
@@ -14,20 +14,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.metadata;
+package org.apache.calcite.rel.metadata;
 
-import org.eigenbase.rel.RelNode;
+import org.apache.calcite.rel.RelNode;
 
 /**
  * Metadata about a relational expression.
  *
  * <p>For particular types of metadata, a sub-class defines one of more methods
  * to query that metadata. Then a {@link RelMetadataProvider} can offer those
- * kinds of metadata for particular sub-classes of {@link RelNode}.</p>
+ * kinds of metadata for particular sub-classes of {@link RelNode}.
  *
  * <p>User code (typically in a planner rule or an implementation of
- * {@link RelNode#computeSelfCost(org.eigenbase.relopt.RelOptPlanner)}) acquires
- * a {@code Metadata} instance by calling {@link RelNode#metadata}.</p>
+ * {@link RelNode#computeSelfCost(org.apache.calcite.plan.RelOptPlanner)})
+ * acquires a {@code Metadata} instance by calling {@link RelNode#metadata}.
  *
  * <p>A {@code Metadata} instance already knows which particular {@code RelNode}
  * it is describing, so the methods do not pass in the {@code RelNode}. In fact,

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/metadata/MetadataFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/MetadataFactory.java b/core/src/main/java/org/apache/calcite/rel/metadata/MetadataFactory.java
index 428d533..7651ecd 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/MetadataFactory.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/MetadataFactory.java
@@ -14,9 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.metadata;
+package org.apache.calcite.rel.metadata;
 
-import org.eigenbase.rel.RelNode;
+import org.apache.calcite.rel.RelNode;
 
 /**
  * Source of metadata about relational expressions.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 9512597..e20d6ce 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
@@ -14,12 +14,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.metadata;
+package org.apache.calcite.rel.metadata;
 
-import java.util.concurrent.ExecutionException;
-
-import org.eigenbase.rel.RelNode;
-import org.eigenbase.util.Pair;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.util.Pair;
 
 import com.google.common.base.Function;
 import com.google.common.base.Functions;
@@ -27,6 +25,8 @@ import com.google.common.cache.CacheBuilder;
 import com.google.common.cache.CacheLoader;
 import com.google.common.cache.LoadingCache;
 
+import java.util.concurrent.ExecutionException;
+
 /** Implementation of {@link MetadataFactory} that gets providers from a
  * {@link RelMetadataProvider} and stores them in a cache.
  *
@@ -51,8 +51,7 @@ public class MetadataFactoryImpl implements MetadataFactory {
       Function<RelNode, Metadata>> loader(final RelMetadataProvider provider) {
     return new CacheLoader<Pair<Class<RelNode>, Class<Metadata>>,
         Function<RelNode, Metadata>>() {
-      @Override
-      public Function<RelNode, Metadata> load(
+      @Override public Function<RelNode, Metadata> load(
           Pair<Class<RelNode>, Class<Metadata>> key) throws Exception {
         final Function<RelNode, Metadata> function =
             provider.apply(key.left, key.right);


[51/58] [abbrv] [partial] incubator-calcite git commit: [CALCITE-306] Standardize code style for "import package.*; "

Posted by jh...@apache.org.
[CALCITE-306] Standardize code style for "import package.*;"

Rename classes and packages and generally fix things up after [CALCITE-296] and [CALCITE-419].

Make many inner classes of EnumerableRules (e.g. EnumerableFilter, EnumerableFilterRule) top-level classes.

Apply a consistent formatting convention for string literals split over multiple lines.

Rename generated file org.eigenbase.resource.Resources to org.apache.calcite.runtime.Resources.

Change occurrences of "optiq" and "eigenbase" in java code to "calcite".


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

Branch: refs/heads/master
Commit: a0ba73cd2de76696b96a1cd828d2aa4d3ef9eb55
Parents: a611d64
Author: Julian Hyde <jh...@apache.org>
Authored: Thu Nov 13 18:11:34 2014 -0800
Committer: Julian Hyde <jh...@apache.org>
Committed: Thu Nov 13 18:22:06 2014 -0800

----------------------------------------------------------------------
 avatica/pom.xml                                 |    4 +-
 .../org/apache/calcite/avatica/ArrayImpl.java   |   12 +-
 .../calcite/avatica/AvaticaConnection.java      |   31 +-
 .../avatica/AvaticaDatabaseMetaData.java        |   23 +-
 .../apache/calcite/avatica/AvaticaFactory.java  |    2 +-
 .../calcite/avatica/AvaticaJdbc40Factory.java   |    2 +-
 .../calcite/avatica/AvaticaJdbc41Factory.java   |    8 +-
 .../calcite/avatica/AvaticaParameter.java       |   13 +-
 .../calcite/avatica/AvaticaPrepareResult.java   |    2 +-
 .../avatica/AvaticaPreparedStatement.java       |   15 +-
 .../calcite/avatica/AvaticaResultSet.java       |   23 +-
 .../avatica/AvaticaResultSetMetaData.java       |    2 +-
 .../calcite/avatica/AvaticaStatement.java       |   12 +-
 .../avatica/BuiltInConnectionProperty.java      |    5 +-
 .../org/apache/calcite/avatica/ByteString.java  |   14 +-
 .../java/org/apache/calcite/avatica/Casing.java |    2 +-
 .../apache/calcite/avatica/ColumnMetaData.java  |   10 +-
 .../calcite/avatica/ConnectStringParser.java    |   16 +-
 .../calcite/avatica/ConnectionConfig.java       |    2 +-
 .../calcite/avatica/ConnectionConfigImpl.java   |    4 +-
 .../calcite/avatica/ConnectionProperty.java     |    2 +-
 .../java/org/apache/calcite/avatica/Cursor.java |   19 +-
 .../apache/calcite/avatica/DriverVersion.java   |    2 +-
 .../org/apache/calcite/avatica/Handler.java     |    2 +-
 .../org/apache/calcite/avatica/HandlerImpl.java |    2 +-
 .../java/org/apache/calcite/avatica/Helper.java |    2 +-
 .../calcite/avatica/InternalProperty.java       |    8 +-
 .../java/org/apache/calcite/avatica/Meta.java   |    2 +-
 .../org/apache/calcite/avatica/Quoting.java     |    2 +-
 .../calcite/avatica/UnregisteredDriver.java     |   26 +-
 .../apache/calcite/avatica/package-info.java    |    2 +-
 .../calcite/avatica/test/AvaticaSuite.java      |    2 +-
 .../avatica/test/ConnectStringParserTest.java   |   18 +-
 .../calcite/avatica/test/package-info.java      |    2 +-
 core/pom.xml                                    |    6 +-
 core/src/main/codegen/config.fmpp               |    8 +-
 core/src/main/codegen/templates/Parser.jj       |  110 +-
 .../java/org/apache/calcite/DataContext.java    |   19 +-
 core/src/main/java/org/apache/calcite/Demo.java |    2 +-
 .../calcite/adapter/clone/ArrayTable.java       |   45 +-
 .../calcite/adapter/clone/CloneSchema.java      |   39 +-
 .../calcite/adapter/clone/ColumnLoader.java     |   37 +-
 .../apache/calcite/adapter/clone/ListTable.java |   29 +-
 .../calcite/adapter/clone/package-info.java     |    2 +-
 .../adapter/enumerable/AggAddContext.java       |   22 +-
 .../calcite/adapter/enumerable/AggContext.java  |   30 +-
 .../calcite/adapter/enumerable/AggImpState.java |    9 +-
 .../adapter/enumerable/AggImplementor.java      |   17 +-
 .../adapter/enumerable/AggResetContext.java     |   12 +-
 .../adapter/enumerable/AggResultContext.java    |   12 +-
 .../adapter/enumerable/CallImplementor.java     |   17 +-
 .../calcite/adapter/enumerable/EnumUtils.java   |  115 +
 .../adapter/enumerable/EnumerableAggregate.java |  425 +++
 .../enumerable/EnumerableAggregateRule.java     |   54 +
 .../adapter/enumerable/EnumerableCalc.java      |  221 ++
 .../adapter/enumerable/EnumerableCalcRule.java  |   61 +
 .../adapter/enumerable/EnumerableCollect.java   |   71 +
 .../enumerable/EnumerableCollectRule.java       |   49 +
 .../enumerable/EnumerableConvention.java        |   13 +-
 .../adapter/enumerable/EnumerableEmptyRule.java |   48 +
 .../adapter/enumerable/EnumerableFilter.java    |   50 +
 .../enumerable/EnumerableFilterRule.java        |   56 +
 .../enumerable/EnumerableFilterToCalcRule.java  |   62 +
 .../enumerable/EnumerableInterpreter.java       |   79 +
 .../adapter/enumerable/EnumerableIntersect.java |   83 +
 .../enumerable/EnumerableIntersectRule.java     |   48 +
 .../adapter/enumerable/EnumerableJoin.java      |  221 ++
 .../adapter/enumerable/EnumerableJoinRule.java  |   89 +
 .../adapter/enumerable/EnumerableLimit.java     |  104 +
 .../adapter/enumerable/EnumerableLimitRule.java |   68 +
 .../adapter/enumerable/EnumerableMinus.java     |   81 +
 .../adapter/enumerable/EnumerableMinusRule.java |   49 +
 .../enumerable/EnumerableOneRowRule.java        |   50 +
 .../adapter/enumerable/EnumerableProject.java   |   54 +
 .../enumerable/EnumerableProjectRule.java       |   59 +
 .../enumerable/EnumerableProjectToCalcRule.java |   57 +
 .../adapter/enumerable/EnumerableRel.java       |   31 +-
 .../enumerable/EnumerableRelImplementor.java    |   58 +-
 .../adapter/enumerable/EnumerableRules.java     | 3306 +-----------------
 .../adapter/enumerable/EnumerableSemiJoin.java  |  114 +
 .../enumerable/EnumerableSemiJoinRule.java      |   64 +
 .../adapter/enumerable/EnumerableSort.java      |   80 +
 .../adapter/enumerable/EnumerableSortRule.java  |   55 +
 .../enumerable/EnumerableTableFunctionScan.java |   73 +
 .../EnumerableTableFunctionScanRule.java        |   45 +
 .../enumerable/EnumerableTableModify.java       |  149 +
 .../enumerable/EnumerableTableModifyRule.java   |   57 +
 .../adapter/enumerable/EnumerableTableScan.java |  153 +
 .../adapter/enumerable/EnumerableUncollect.java |   72 +
 .../enumerable/EnumerableUncollectRule.java     |   48 +
 .../adapter/enumerable/EnumerableUnion.java     |   82 +
 .../adapter/enumerable/EnumerableUnionRule.java |   44 +
 .../adapter/enumerable/EnumerableValues.java    |   96 +
 .../enumerable/EnumerableValuesRule.java        |   44 +
 .../adapter/enumerable/EnumerableWindow.java    |  952 +++++
 .../enumerable/EnumerableWindowRule.java        |   48 +
 .../adapter/enumerable/JavaRelImplementor.java  |   19 +-
 .../adapter/enumerable/JavaRowFormat.java       |   58 +-
 .../adapter/enumerable/NestedBlockBuilder.java  |   14 +-
 .../enumerable/NestedBlockBuilderImpl.java      |   22 +-
 .../adapter/enumerable/NotNullImplementor.java  |   14 +-
 .../calcite/adapter/enumerable/NullPolicy.java  |    4 +-
 .../calcite/adapter/enumerable/PhysType.java    |   17 +-
 .../adapter/enumerable/PhysTypeImpl.java        |   82 +-
 .../ReflectiveCallNotNullImplementor.java       |   13 +-
 .../calcite/adapter/enumerable/RexImpTable.java |  355 +-
 .../adapter/enumerable/RexToLixTranslator.java  |  112 +-
 .../enumerable/StrictAggImplementor.java        |   29 +-
 .../enumerable/StrictWinAggImplementor.java     |   28 +-
 .../adapter/enumerable/WinAggAddContext.java    |   10 +-
 .../adapter/enumerable/WinAggContext.java       |    4 +-
 .../adapter/enumerable/WinAggFrameContext.java  |    8 +-
 .../enumerable/WinAggFrameResultContext.java    |    4 +-
 .../adapter/enumerable/WinAggImplementor.java   |   10 +-
 .../adapter/enumerable/WinAggResetContext.java  |   11 +-
 .../adapter/enumerable/WinAggResultContext.java |   21 +-
 .../enumerable/impl/AggAddContextImpl.java      |   14 +-
 .../enumerable/impl/AggResetContextImpl.java    |   16 +-
 .../enumerable/impl/AggResultContextImpl.java   |   14 +-
 .../enumerable/impl/WinAggAddContextImpl.java   |   22 +-
 .../enumerable/impl/WinAggResetContextImpl.java |   13 +-
 .../impl/WinAggResultContextImpl.java           |   20 +-
 .../adapter/enumerable/impl/package-info.java   |    4 +-
 .../adapter/enumerable/package-info.java        |    2 +-
 .../adapter/java/AbstractQueryableTable.java    |   13 +-
 .../calcite/adapter/java/JavaTypeFactory.java   |    6 +-
 .../calcite/adapter/java/ReflectiveSchema.java  |   76 +-
 .../calcite/adapter/java/package-info.java      |    2 +-
 .../calcite/adapter/jdbc/JdbcConvention.java    |   24 +-
 .../calcite/adapter/jdbc/JdbcImplementor.java   |   76 +-
 .../calcite/adapter/jdbc/JdbcQueryProvider.java |    7 +-
 .../apache/calcite/adapter/jdbc/JdbcRel.java    |    4 +-
 .../apache/calcite/adapter/jdbc/JdbcRules.java  |  440 +--
 .../apache/calcite/adapter/jdbc/JdbcSchema.java |   53 +-
 .../apache/calcite/adapter/jdbc/JdbcTable.java  |   67 +-
 .../calcite/adapter/jdbc/JdbcTableScan.java     |   15 +-
 .../adapter/jdbc/JdbcToEnumerableConverter.java |   87 +-
 .../jdbc/JdbcToEnumerableConverterRule.java     |   11 +-
 .../apache/calcite/adapter/jdbc/JdbcUtils.java  |   36 +-
 .../calcite/adapter/jdbc/package-info.java      |    2 +-
 .../apache/calcite/adapter/package-info.java    |   23 +-
 .../calcite/config/CalciteConnectionConfig.java |   34 +-
 .../config/CalciteConnectionConfigImpl.java     |   44 +-
 .../config/CalciteConnectionProperty.java       |   27 +-
 .../java/org/apache/calcite/config/Lex.java     |    6 +-
 .../org/apache/calcite/config/package-info.java |    2 +-
 .../org/apache/calcite/interpreter/Context.java |    2 +-
 .../apache/calcite/interpreter/FilterNode.java  |    9 +-
 .../apache/calcite/interpreter/Interpreter.java |   40 +-
 .../org/apache/calcite/interpreter/Node.java    |    2 +-
 .../org/apache/calcite/interpreter/Nodes.java   |   92 +-
 .../apache/calcite/interpreter/ProjectNode.java |   11 +-
 .../org/apache/calcite/interpreter/Row.java     |    2 +-
 .../org/apache/calcite/interpreter/Scalar.java  |    2 +-
 .../apache/calcite/interpreter/ScanNode.java    |   47 +-
 .../org/apache/calcite/interpreter/Sink.java    |    2 +-
 .../apache/calcite/interpreter/SortNode.java    |   16 +-
 .../org/apache/calcite/interpreter/Source.java  |    2 +-
 .../apache/calcite/interpreter/ValuesNode.java  |   13 +-
 .../calcite/interpreter/package-info.java       |    2 +-
 .../apache/calcite/jdbc/CalciteConnection.java  |   19 +-
 .../calcite/jdbc/CalciteConnectionImpl.java     |  186 +-
 .../org/apache/calcite/jdbc/CalciteFactory.java |   19 +-
 .../calcite/jdbc/CalciteJdbc40Factory.java      |   10 +-
 .../calcite/jdbc/CalciteJdbc41Factory.java      |   89 +-
 .../org/apache/calcite/jdbc/CalcitePrepare.java |   84 +-
 .../calcite/jdbc/CalcitePreparedStatement.java  |   35 +-
 .../apache/calcite/jdbc/CalciteResultSet.java   |   57 +-
 .../apache/calcite/jdbc/CalciteRootSchema.java  |   10 +-
 .../org/apache/calcite/jdbc/CalciteSchema.java  |  202 +-
 .../apache/calcite/jdbc/CalciteStatement.java   |   40 +-
 .../java/org/apache/calcite/jdbc/Driver.java    |   67 +-
 .../org/apache/calcite/jdbc/JavaRecordType.java |    6 +-
 .../calcite/jdbc/JavaTypeFactoryImpl.java       |   39 +-
 .../java/org/apache/calcite/jdbc/MetaImpl.java  |  150 +-
 .../org/apache/calcite/jdbc/MetadataSchema.java |   20 +-
 .../calcite/jdbc/SqlTimeoutException.java       |    2 +-
 .../org/apache/calcite/jdbc/package-info.java   |    2 +-
 .../org/apache/calcite/materialize/Lattice.java |  115 +-
 .../materialize/MaterializationActor.java       |   31 +-
 .../calcite/materialize/MaterializationKey.java |   11 +-
 .../materialize/MaterializationService.java     |  116 +-
 .../org/apache/calcite/materialize/TileKey.java |    9 +-
 .../calcite/materialize/TileSuggester.java      |    2 +-
 .../calcite/materialize/package-info.java       |    6 +-
 .../org/apache/calcite/model/JsonColumn.java    |    2 +-
 .../apache/calcite/model/JsonCustomSchema.java  |    9 +-
 .../apache/calcite/model/JsonCustomTable.java   |    4 +-
 .../org/apache/calcite/model/JsonFunction.java  |    2 +-
 .../apache/calcite/model/JsonJdbcSchema.java    |    5 +-
 .../org/apache/calcite/model/JsonLattice.java   |    2 +-
 .../org/apache/calcite/model/JsonMapSchema.java |    8 +-
 .../calcite/model/JsonMaterialization.java      |    5 +-
 .../org/apache/calcite/model/JsonMeasure.java   |    6 +-
 .../java/org/apache/calcite/model/JsonRoot.java |    4 +-
 .../org/apache/calcite/model/JsonSchema.java    |    7 +-
 .../org/apache/calcite/model/JsonTable.java     |    2 +-
 .../java/org/apache/calcite/model/JsonTile.java |    4 +-
 .../java/org/apache/calcite/model/JsonView.java |    5 +-
 .../org/apache/calcite/model/ModelHandler.java  |   77 +-
 .../org/apache/calcite/model/package-info.java  |    8 +-
 .../java/org/apache/calcite/package-info.java   |    2 +-
 .../calcite/plan/AbstractRelOptPlanner.java     |   38 +-
 .../calcite/plan/CommonRelSubExprRule.java      |    2 +-
 .../java/org/apache/calcite/plan/Context.java   |    4 +-
 .../java/org/apache/calcite/plan/Contexts.java  |   14 +-
 .../org/apache/calcite/plan/Convention.java     |    4 +-
 .../apache/calcite/plan/ConventionTraitDef.java |   24 +-
 .../calcite/plan/MulticastRelOptListener.java   |    5 +-
 .../org/apache/calcite/plan/RelImplementor.java |   13 +-
 .../calcite/plan/RelOptAbstractTable.java       |   19 +-
 .../org/apache/calcite/plan/RelOptCluster.java  |   20 +-
 .../apache/calcite/plan/RelOptConnection.java   |    2 +-
 .../org/apache/calcite/plan/RelOptCost.java     |    6 +-
 .../apache/calcite/plan/RelOptCostFactory.java  |    2 +-
 .../org/apache/calcite/plan/RelOptCostImpl.java |    7 +-
 .../org/apache/calcite/plan/RelOptLattice.java  |   27 +-
 .../org/apache/calcite/plan/RelOptListener.java |    7 +-
 .../calcite/plan/RelOptMaterialization.java     |  109 +-
 .../org/apache/calcite/plan/RelOptNode.java     |    6 +-
 .../org/apache/calcite/plan/RelOptPlanner.java  |   55 +-
 .../calcite/plan/RelOptPredicateList.java       |    6 +-
 .../org/apache/calcite/plan/RelOptQuery.java    |   16 +-
 .../org/apache/calcite/plan/RelOptRule.java     |   28 +-
 .../org/apache/calcite/plan/RelOptRuleCall.java |   37 +-
 .../apache/calcite/plan/RelOptRuleOperand.java  |   27 +-
 .../plan/RelOptRuleOperandChildPolicy.java      |    2 +-
 .../calcite/plan/RelOptRuleOperandChildren.java |    6 +-
 .../calcite/plan/RelOptSamplingParameters.java  |    6 +-
 .../org/apache/calcite/plan/RelOptSchema.java   |    6 +-
 .../calcite/plan/RelOptSchemaWithSampling.java  |    2 +-
 .../org/apache/calcite/plan/RelOptTable.java    |   22 +-
 .../org/apache/calcite/plan/RelOptUtil.java     |  324 +-
 .../java/org/apache/calcite/plan/RelTrait.java  |    8 +-
 .../org/apache/calcite/plan/RelTraitDef.java    |   26 +-
 .../plan/RelTraitPropagationVisitor.java        |    7 +-
 .../org/apache/calcite/plan/RelTraitSet.java    |   24 +-
 .../java/org/apache/calcite/plan/Strong.java    |   12 +-
 .../calcite/plan/SubstitutionVisitor.java       |  275 +-
 .../org/apache/calcite/plan/TableAccessMap.java |   19 +-
 .../apache/calcite/plan/VisitorRelVisitor.java  |   14 +-
 .../apache/calcite/plan/hep/HepInstruction.java |   18 +-
 .../apache/calcite/plan/hep/HepMatchOrder.java  |    2 +-
 .../org/apache/calcite/plan/hep/HepPlanner.java |   69 +-
 .../org/apache/calcite/plan/hep/HepProgram.java |   14 +-
 .../calcite/plan/hep/HepProgramBuilder.java     |   28 +-
 .../plan/hep/HepRelMetadataProvider.java        |    7 +-
 .../apache/calcite/plan/hep/HepRelVertex.java   |   39 +-
 .../apache/calcite/plan/hep/HepRuleCall.java    |   13 +-
 .../apache/calcite/plan/hep/package-info.java   |    4 +-
 .../org/apache/calcite/plan/package-info.java   |    2 +-
 .../calcite/plan/volcano/AbstractConverter.java |   32 +-
 .../ChainedPhaseRuleMappingInitializer.java     |   13 +-
 .../org/apache/calcite/plan/volcano/RelSet.java |   28 +-
 .../apache/calcite/plan/volcano/RelSubset.java  |   56 +-
 .../apache/calcite/plan/volcano/RuleQueue.java  |   49 +-
 .../calcite/plan/volcano/VolcanoCost.java       |   13 +-
 .../calcite/plan/volcano/VolcanoPlanner.java    |  198 +-
 .../plan/volcano/VolcanoPlannerPhase.java       |    8 +-
 ...lcanoPlannerPhaseRuleMappingInitializer.java |   17 +-
 .../volcano/VolcanoRelMetadataProvider.java     |    7 +-
 .../calcite/plan/volcano/VolcanoRuleCall.java   |   32 +-
 .../calcite/plan/volcano/VolcanoRuleMatch.java  |    8 +-
 .../calcite/plan/volcano/package-info.java      |   89 +-
 .../calcite/prepare/CalciteCatalogReader.java   |   93 +-
 .../calcite/prepare/CalciteMaterializer.java    |   96 +-
 .../calcite/prepare/CalcitePrepareImpl.java     |  329 +-
 .../calcite/prepare/CalciteSqlValidator.java    |   29 +-
 .../calcite/prepare/LixToRelTranslator.java     |   55 +-
 .../org/apache/calcite/prepare/PlannerImpl.java |   89 +-
 .../org/apache/calcite/prepare/Prepare.java     |  112 +-
 .../calcite/prepare/QueryableRelBuilder.java    |   61 +-
 .../apache/calcite/prepare/RelOptTableImpl.java |   51 +-
 .../apache/calcite/prepare/package-info.java    |    2 +-
 .../org/apache/calcite/rel/AbstractRelNode.java |   73 +-
 .../org/apache/calcite/rel/Aggregation.java     |   70 -
 .../apache/calcite/rel/InvalidRelException.java |    2 +-
 .../org/apache/calcite/rel/RelCollation.java    |    6 +-
 .../apache/calcite/rel/RelCollationImpl.java    |   17 +-
 .../calcite/rel/RelCollationTraitDef.java       |   21 +-
 .../apache/calcite/rel/RelFieldCollation.java   |    2 +-
 .../apache/calcite/rel/RelImplementorImpl.java  |   19 +-
 .../java/org/apache/calcite/rel/RelInput.java   |   15 +-
 .../java/org/apache/calcite/rel/RelNode.java    |   72 +-
 .../java/org/apache/calcite/rel/RelShuttle.java |   39 +-
 .../org/apache/calcite/rel/RelShuttleImpl.java  |   53 +-
 .../java/org/apache/calcite/rel/RelVisitor.java |   10 +-
 .../java/org/apache/calcite/rel/RelWriter.java  |   13 +-
 .../java/org/apache/calcite/rel/SingleRel.java  |   59 +-
 .../apache/calcite/rel/convert/Converter.java   |   27 +-
 .../calcite/rel/convert/ConverterImpl.java      |   31 +-
 .../calcite/rel/convert/ConverterRule.java      |   15 +-
 .../calcite/rel/convert/NoneConverter.java      |   24 +-
 .../calcite/rel/convert/TraitMatchingRule.java  |   11 +-
 .../calcite/rel/convert/package-info.java       |    2 +-
 .../org/apache/calcite/rel/core/Aggregate.java  |   98 +-
 .../apache/calcite/rel/core/AggregateCall.java  |   72 +-
 .../java/org/apache/calcite/rel/core/Calc.java  |   53 +-
 .../org/apache/calcite/rel/core/Collect.java    |   49 +-
 .../apache/calcite/rel/core/Correlation.java    |    2 +-
 .../org/apache/calcite/rel/core/Correlator.java |   53 +-
 .../java/org/apache/calcite/rel/core/Empty.java |   52 +-
 .../org/apache/calcite/rel/core/Filter.java     |   63 +-
 .../org/apache/calcite/rel/core/Intersect.java  |   39 +-
 .../java/org/apache/calcite/rel/core/Join.java  |   77 +-
 .../org/apache/calcite/rel/core/JoinInfo.java   |   31 +-
 .../apache/calcite/rel/core/JoinRelType.java    |    2 +-
 .../java/org/apache/calcite/rel/core/Minus.java |   43 +-
 .../org/apache/calcite/rel/core/OneRow.java     |   38 +-
 .../org/apache/calcite/rel/core/Project.java    |   99 +-
 .../apache/calcite/rel/core/RelFactories.java   |   85 +-
 .../org/apache/calcite/rel/core/Sample.java     |   52 +-
 .../org/apache/calcite/rel/core/SemiJoin.java   |   61 +-
 .../java/org/apache/calcite/rel/core/SetOp.java |   85 +-
 .../java/org/apache/calcite/rel/core/Sort.java  |   73 +-
 .../calcite/rel/core/TableFunctionScan.java     |   62 +-
 .../apache/calcite/rel/core/TableModify.java    |   49 +-
 .../org/apache/calcite/rel/core/TableScan.java  |   52 +-
 .../org/apache/calcite/rel/core/Uncollect.java  |   38 +-
 .../java/org/apache/calcite/rel/core/Union.java |   29 +-
 .../org/apache/calcite/rel/core/Values.java     |   59 +-
 .../org/apache/calcite/rel/core/Window.java     |  136 +-
 .../apache/calcite/rel/core/package-info.java   |   36 +
 .../apache/calcite/rel/externalize/RelJson.java |   75 +-
 .../calcite/rel/externalize/RelJsonReader.java  |   56 +-
 .../calcite/rel/externalize/RelJsonWriter.java  |   18 +-
 .../calcite/rel/externalize/RelWriterImpl.java  |   27 +-
 .../calcite/rel/externalize/RelXmlWriter.java   |   14 +-
 .../calcite/rel/externalize/package-info.java   |   24 +
 .../apache/calcite/rel/jdbc/package-info.java   |    2 +-
 .../calcite/rel/logical/LogicalAggregate.java   |   38 +-
 .../apache/calcite/rel/logical/LogicalCalc.java |   55 +-
 .../calcite/rel/logical/LogicalFilter.java      |   46 +-
 .../calcite/rel/logical/LogicalIntersect.java   |   42 +-
 .../apache/calcite/rel/logical/LogicalJoin.java |   71 +-
 .../calcite/rel/logical/LogicalMinus.java       |   36 +-
 .../calcite/rel/logical/LogicalOneRow.java      |   33 +-
 .../calcite/rel/logical/LogicalProject.java     |   56 +-
 .../rel/logical/LogicalTableFunctionScan.java   |   42 +-
 .../calcite/rel/logical/LogicalTableModify.java |   32 +-
 .../calcite/rel/logical/LogicalTableScan.java   |   30 +-
 .../calcite/rel/logical/LogicalUnion.java       |   33 +-
 .../calcite/rel/logical/LogicalValues.java      |   51 +-
 .../calcite/rel/logical/LogicalWindow.java      |  144 +-
 .../calcite/rel/logical/package-info.java       |   36 +
 .../calcite/rel/metadata/BuiltInMetadata.java   |   21 +-
 .../metadata/CachingRelMetadataProvider.java    |   19 +-
 .../metadata/ChainedRelMetadataProvider.java    |   14 +-
 .../metadata/DefaultRelMetadataProvider.java    |   10 +-
 .../apache/calcite/rel/metadata/Metadata.java   |   10 +-
 .../calcite/rel/metadata/MetadataFactory.java   |    4 +-
 .../rel/metadata/MetadataFactoryImpl.java       |   13 +-
 .../metadata/ReflectiveRelMetadataProvider.java |   40 +-
 .../calcite/rel/metadata/RelColumnMapping.java  |    6 +-
 .../calcite/rel/metadata/RelColumnOrigin.java   |    6 +-
 .../rel/metadata/RelMdColumnOrigins.java        |   55 +-
 .../rel/metadata/RelMdColumnUniqueness.java     |   58 +-
 .../rel/metadata/RelMdDistinctRowCount.java     |   63 +-
 .../rel/metadata/RelMdExplainVisibility.java    |   11 +-
 .../metadata/RelMdPercentageOriginalRows.java   |   36 +-
 .../rel/metadata/RelMdPopulationSize.java       |   57 +-
 .../calcite/rel/metadata/RelMdPredicates.java   |  133 +-
 .../calcite/rel/metadata/RelMdRowCount.java     |   50 +-
 .../calcite/rel/metadata/RelMdSelectivity.java  |   59 +-
 .../calcite/rel/metadata/RelMdUniqueKeys.java   |   60 +-
 .../apache/calcite/rel/metadata/RelMdUtil.java  |   63 +-
 .../rel/metadata/RelMetadataProvider.java       |    8 +-
 .../calcite/rel/metadata/RelMetadataQuery.java  |   48 +-
 .../calcite/rel/metadata/package-info.java      |    2 +-
 .../org/apache/calcite/rel/package-info.java    |   27 +-
 .../AggregateExpandDistinctAggregatesRule.java  |   99 +-
 .../rel/rules/AggregateFilterTransposeRule.java |   61 +-
 .../rel/rules/AggregateProjectMergeRule.java    |   51 +-
 .../AggregateProjectPullUpConstantsRule.java    |   69 +-
 .../rel/rules/AggregateReduceFunctionsRule.java |  127 +-
 .../calcite/rel/rules/AggregateRemoveRule.java  |   34 +-
 .../rel/rules/AggregateStarTableRule.java       |  105 +-
 .../rel/rules/AggregateUnionAggregateRule.java  |   73 +-
 .../rel/rules/AggregateUnionTransposeRule.java  |   74 +-
 .../apache/calcite/rel/rules/CalcMergeRule.java |   42 +-
 .../calcite/rel/rules/CalcRelSplitter.java      |  104 +-
 .../calcite/rel/rules/CalcRemoveRule.java       |   32 +-
 .../calcite/rel/rules/CoerceInputsRule.java     |   13 +-
 .../calcite/rel/rules/EmptyPruneRules.java      |  103 +-
 .../org/apache/calcite/rel/rules/EquiJoin.java  |   33 +-
 .../rel/rules/FilterAggregateTransposeRule.java |   43 +-
 .../calcite/rel/rules/FilterCalcMergeRule.java  |   50 +-
 .../calcite/rel/rules/FilterJoinRule.java       |   97 +-
 .../calcite/rel/rules/FilterMergeRule.java      |   53 +-
 .../rel/rules/FilterMultiJoinMergeRule.java     |   63 +-
 .../rel/rules/FilterProjectTransposeRule.java   |   49 +-
 .../FilterRemoveIsNotDistinctFromRule.java      |   46 +-
 .../rel/rules/FilterSetOpTransposeRule.java     |   52 +-
 .../rules/FilterTableFunctionTransposeRule.java |   62 +-
 .../calcite/rel/rules/FilterTableRule.java      |   64 +-
 .../calcite/rel/rules/FilterToCalcRule.java     |   43 +-
 .../rel/rules/JoinAddRedundantSemiJoinRule.java |   34 +-
 .../calcite/rel/rules/JoinAssociateRule.java    |   67 +-
 .../calcite/rel/rules/JoinCommuteRule.java      |   72 +-
 .../rel/rules/JoinExtractFilterRule.java        |   36 +-
 .../rel/rules/JoinProjectTransposeRule.java     |  115 +-
 .../rel/rules/JoinPushThroughJoinRule.java      |   73 +-
 .../rules/JoinPushTransitivePredicatesRule.java |   47 +-
 .../calcite/rel/rules/JoinToCorrelatorRule.java |   54 +-
 .../calcite/rel/rules/JoinToMultiJoinRule.java  |  173 +-
 .../rel/rules/JoinUnionTransposeRule.java       |   52 +-
 .../apache/calcite/rel/rules/LoptJoinTree.java  |   17 +-
 .../apache/calcite/rel/rules/LoptMultiJoin.java |   68 +-
 .../calcite/rel/rules/LoptOptimizeJoinRule.java |  101 +-
 .../rel/rules/LoptSemiJoinOptimizer.java        |   92 +-
 .../org/apache/calcite/rel/rules/MultiJoin.java |   55 +-
 .../rel/rules/MultiJoinOptimizeBushyRule.java   |   67 +-
 .../rules/MultiJoinProjectTransposeRule.java    |  106 +-
 .../calcite/rel/rules/ProjectCalcMergeRule.java |   61 +-
 .../rel/rules/ProjectFilterTransposeRule.java   |   44 +-
 .../rel/rules/ProjectJoinTransposeRule.java     |   43 +-
 .../calcite/rel/rules/ProjectMergeRule.java     |   72 +-
 .../rel/rules/ProjectMultiJoinMergeRule.java    |   47 +-
 .../calcite/rel/rules/ProjectRemoveRule.java    |   68 +-
 .../rel/rules/ProjectSetOpTransposeRule.java    |   58 +-
 .../rel/rules/ProjectSortTransposeRule.java     |   43 +-
 .../calcite/rel/rules/ProjectTableRule.java     |   89 +-
 .../calcite/rel/rules/ProjectToCalcRule.java    |   37 +-
 .../calcite/rel/rules/ProjectToWindowRule.java  |  133 +-
 .../apache/calcite/rel/rules/PushProjector.java |   57 +-
 .../calcite/rel/rules/ReduceDecimalsRule.java   |  109 +-
 .../rel/rules/ReduceExpressionsRule.java        |  118 +-
 .../rel/rules/SemiJoinFilterTransposeRule.java  |   56 +-
 .../rel/rules/SemiJoinJoinTransposeRule.java    |   61 +-
 .../rel/rules/SemiJoinProjectTransposeRule.java |   95 +-
 .../calcite/rel/rules/SemiJoinRemoveRule.java   |   37 +-
 .../apache/calcite/rel/rules/SemiJoinRule.java  |   63 +-
 .../rel/rules/SortProjectTransposeRule.java     |   55 +-
 .../calcite/rel/rules/SortRemoveRule.java       |   37 +-
 .../apache/calcite/rel/rules/TableScanRule.java |   27 +-
 .../calcite/rel/rules/UnionEliminatorRule.java  |   13 +-
 .../calcite/rel/rules/UnionMergeRule.java       |   71 +-
 .../calcite/rel/rules/UnionToDistinctRule.java  |   25 +-
 .../calcite/rel/rules/ValuesReduceRule.java     |   97 +-
 .../apache/calcite/rel/rules/package-info.java  |   30 +-
 .../apache/calcite/rel/type/RelCrossType.java   |   18 +-
 .../apache/calcite/rel/type/RelDataType.java    |   18 +-
 .../rel/type/RelDataTypeComparability.java      |    4 +-
 .../calcite/rel/type/RelDataTypeFactory.java    |   24 +-
 .../rel/type/RelDataTypeFactoryImpl.java        |   55 +-
 .../calcite/rel/type/RelDataTypeFamily.java     |    2 +-
 .../calcite/rel/type/RelDataTypeField.java      |    6 +-
 .../calcite/rel/type/RelDataTypeFieldImpl.java  |   10 +-
 .../calcite/rel/type/RelDataTypeImpl.java       |   39 +-
 .../rel/type/RelDataTypePrecedenceList.java     |    2 +-
 .../calcite/rel/type/RelDataTypeSystem.java     |    6 +-
 .../calcite/rel/type/RelDataTypeSystemImpl.java |    7 +-
 .../calcite/rel/type/RelProtoDataType.java      |    8 +-
 .../apache/calcite/rel/type/RelRecordType.java  |   21 +-
 .../apache/calcite/rel/type/package-info.java   |    2 +-
 .../java/org/apache/calcite/rex/RexAction.java  |    2 +-
 .../java/org/apache/calcite/rex/RexBuilder.java |   67 +-
 .../java/org/apache/calcite/rex/RexCall.java    |   21 +-
 .../org/apache/calcite/rex/RexCallBinding.java  |   23 +-
 .../java/org/apache/calcite/rex/RexChecker.java |   27 +-
 .../java/org/apache/calcite/rex/RexCopier.java  |    2 +-
 .../apache/calcite/rex/RexCorrelVariable.java   |    9 +-
 .../org/apache/calcite/rex/RexDynamicParam.java |    6 +-
 .../org/apache/calcite/rex/RexExecutable.java   |   24 +-
 .../org/apache/calcite/rex/RexExecutorImpl.java |   50 +-
 .../org/apache/calcite/rex/RexFieldAccess.java  |    7 +-
 .../apache/calcite/rex/RexFieldCollation.java   |   15 +-
 .../org/apache/calcite/rex/RexInputRef.java     |   14 +-
 .../java/org/apache/calcite/rex/RexLiteral.java |   61 +-
 .../org/apache/calcite/rex/RexLocalRef.java     |   13 +-
 .../org/apache/calcite/rex/RexMultisetUtil.java |   13 +-
 .../java/org/apache/calcite/rex/RexNode.java    |   18 +-
 .../java/org/apache/calcite/rex/RexOver.java    |   18 +-
 .../java/org/apache/calcite/rex/RexPattern.java |    2 +-
 .../calcite/rex/RexPermutationShuttle.java      |    4 +-
 .../calcite/rex/RexPermuteInputsShuttle.java    |   22 +-
 .../java/org/apache/calcite/rex/RexProgram.java |   59 +-
 .../apache/calcite/rex/RexProgramBuilder.java   |   39 +-
 .../org/apache/calcite/rex/RexRangeRef.java     |   15 +-
 .../java/org/apache/calcite/rex/RexShuttle.java |   14 +-
 .../java/org/apache/calcite/rex/RexSlot.java    |    9 +-
 .../apache/calcite/rex/RexSqlConvertlet.java    |    4 +-
 .../calcite/rex/RexSqlConvertletTable.java      |    2 +-
 .../rex/RexSqlReflectiveConvertletTable.java    |   11 +-
 .../rex/RexSqlStandardConvertletTable.java      |   20 +-
 .../calcite/rex/RexToSqlNodeConverter.java      |    6 +-
 .../calcite/rex/RexToSqlNodeConverterImpl.java  |   13 +-
 .../org/apache/calcite/rex/RexTransformer.java  |   17 +-
 .../java/org/apache/calcite/rex/RexUtil.java    |   63 +-
 .../org/apache/calcite/rex/RexVariable.java     |    4 +-
 .../java/org/apache/calcite/rex/RexVisitor.java |    6 +-
 .../org/apache/calcite/rex/RexVisitorImpl.java  |    4 +-
 .../java/org/apache/calcite/rex/RexWindow.java  |   13 +-
 .../org/apache/calcite/rex/RexWindowBound.java  |   73 +-
 .../org/apache/calcite/rex/package-info.java    |   72 +-
 .../org/apache/calcite/rules/package-info.java  |   23 -
 .../apache/calcite/runtime/AbstractCursor.java  |  119 +-
 .../apache/calcite/runtime/ArrayComparator.java |    2 +-
 .../calcite/runtime/ArrayEnumeratorCursor.java  |    8 +-
 .../apache/calcite/runtime/BinarySearch.java    |   10 +-
 .../org/apache/calcite/runtime/Bindable.java    |    7 +-
 .../runtime/CalciteContextException.java        |   23 +-
 .../calcite/runtime/CalciteException.java       |   20 +-
 .../apache/calcite/runtime/CalciteResource.java |   94 +-
 .../org/apache/calcite/runtime/Enumerables.java |   22 +-
 .../calcite/runtime/EnumeratorCursor.java       |   10 +-
 .../org/apache/calcite/runtime/Feature.java     |    6 +-
 .../org/apache/calcite/runtime/FlatLists.java   |   12 +-
 .../java/org/apache/calcite/runtime/Hook.java   |    2 +-
 .../java/org/apache/calcite/runtime/Like.java   |    2 +-
 .../calcite/runtime/ObjectEnumeratorCursor.java |    8 +-
 .../calcite/runtime/RecordEnumeratorCursor.java |    8 +-
 .../calcite/runtime/ResultSetEnumerable.java    |   22 +-
 .../apache/calcite/runtime/SortedMultiMap.java  |   10 +-
 .../java/org/apache/calcite/runtime/Spacer.java |    2 +-
 .../java/org/apache/calcite/runtime/Spaces.java |   12 +-
 .../apache/calcite/runtime/SqlFunctions.java    |   36 +-
 .../java/org/apache/calcite/runtime/Typed.java  |    4 +-
 .../java/org/apache/calcite/runtime/Unit.java   |    2 +-
 .../org/apache/calcite/runtime/Utilities.java   |    2 +-
 .../apache/calcite/runtime/package-info.java    |    2 +-
 .../calcite/schema/AggregateFunction.java       |    6 +-
 .../apache/calcite/schema/FilterableTable.java  |    8 +-
 .../org/apache/calcite/schema/Function.java     |    2 +-
 .../calcite/schema/FunctionParameter.java       |    6 +-
 .../schema/ImplementableAggFunction.java        |   12 +-
 .../calcite/schema/ImplementableFunction.java   |   10 +-
 .../java/org/apache/calcite/schema/Member.java  |   11 +-
 .../apache/calcite/schema/ModifiableTable.java  |   16 +-
 .../schema/ProjectableFilterableTable.java      |   12 +-
 .../apache/calcite/schema/QueryableTable.java   |    8 +-
 .../apache/calcite/schema/ScalarFunction.java   |    6 +-
 .../apache/calcite/schema/ScannableTable.java   |    5 +-
 .../java/org/apache/calcite/schema/Schema.java  |   11 +-
 .../apache/calcite/schema/SchemaFactory.java    |    6 +-
 .../org/apache/calcite/schema/SchemaPlus.java   |    4 +-
 .../java/org/apache/calcite/schema/Schemas.java |  222 +-
 .../calcite/schema/SemiMutableSchema.java       |    2 +-
 .../org/apache/calcite/schema/Statistic.java    |    2 +-
 .../org/apache/calcite/schema/Statistics.java   |    7 +-
 .../java/org/apache/calcite/schema/Table.java   |   14 +-
 .../org/apache/calcite/schema/TableFactory.java |    4 +-
 .../apache/calcite/schema/TableFunction.java    |   10 +-
 .../org/apache/calcite/schema/TableMacro.java   |    2 +-
 .../calcite/schema/TranslatableTable.java       |    8 +-
 .../calcite/schema/impl/AbstractSchema.java     |   26 +-
 .../calcite/schema/impl/AbstractTable.java      |    7 +-
 .../schema/impl/AbstractTableQueryable.java     |   18 +-
 .../schema/impl/AggregateFunctionImpl.java      |   18 +-
 .../calcite/schema/impl/DelegatingSchema.java   |   17 +-
 .../schema/impl/MaterializedViewTable.java      |   47 +-
 .../schema/impl/ReflectiveFunctionBase.java     |   12 +-
 .../calcite/schema/impl/ScalarFunctionImpl.java |   32 +-
 .../apache/calcite/schema/impl/StarTable.java   |   33 +-
 .../calcite/schema/impl/TableFunctionImpl.java  |   37 +-
 .../calcite/schema/impl/TableMacroImpl.java     |   10 +-
 .../apache/calcite/schema/impl/ViewTable.java   |   51 +-
 .../calcite/schema/impl/package-info.java       |   23 +
 .../org/apache/calcite/schema/package-info.java |   27 +
 .../apache/calcite/server/CalciteServer.java    |   10 +-
 .../calcite/server/CalciteServerStatement.java  |   14 +-
 .../org/apache/calcite/server/package-info.java |    2 +-
 .../calcite/sql/ExplicitOperatorBinding.java    |   17 +-
 .../apache/calcite/sql/JoinConditionType.java   |    4 +-
 .../java/org/apache/calcite/sql/JoinType.java   |    4 +-
 .../calcite/sql/SqlAbstractDateTimeLiteral.java |   28 +-
 .../calcite/sql/SqlAbstractStringLiteral.java   |    8 +-
 .../org/apache/calcite/sql/SqlAccessEnum.java   |    2 +-
 .../org/apache/calcite/sql/SqlAccessType.java   |    4 +-
 .../org/apache/calcite/sql/SqlAggFunction.java  |   13 +-
 .../org/apache/calcite/sql/SqlAsOperator.java   |   21 +-
 .../org/apache/calcite/sql/SqlBasicCall.java    |   11 +-
 .../apache/calcite/sql/SqlBinaryOperator.java   |   27 +-
 .../calcite/sql/SqlBinaryStringLiteral.java     |   12 +-
 .../java/org/apache/calcite/sql/SqlCall.java    |   21 +-
 .../org/apache/calcite/sql/SqlCallBinding.java  |   28 +-
 .../calcite/sql/SqlCharStringLiteral.java       |   14 +-
 .../org/apache/calcite/sql/SqlCollation.java    |   36 +-
 .../org/apache/calcite/sql/SqlDataTypeSpec.java |   30 +-
 .../org/apache/calcite/sql/SqlDateLiteral.java  |   13 +-
 .../java/org/apache/calcite/sql/SqlDelete.java  |   12 +-
 .../java/org/apache/calcite/sql/SqlDialect.java |   24 +-
 .../org/apache/calcite/sql/SqlDynamicParam.java |   10 +-
 .../java/org/apache/calcite/sql/SqlExplain.java |   11 +-
 .../org/apache/calcite/sql/SqlExplainLevel.java |    4 +-
 .../org/apache/calcite/sql/SqlFunction.java     |   25 +-
 .../apache/calcite/sql/SqlFunctionCategory.java |    4 +-
 .../calcite/sql/SqlFunctionalOperator.java      |    6 +-
 .../org/apache/calcite/sql/SqlIdentifier.java   |   23 +-
 .../apache/calcite/sql/SqlInfixOperator.java    |    9 +-
 .../java/org/apache/calcite/sql/SqlInsert.java  |   11 +-
 .../apache/calcite/sql/SqlInsertKeyword.java    |    2 +-
 .../apache/calcite/sql/SqlInternalOperator.java |    6 +-
 .../apache/calcite/sql/SqlIntervalLiteral.java  |    8 +-
 .../calcite/sql/SqlIntervalQualifier.java       |   85 +-
 .../apache/calcite/sql/SqlJdbcFunctionCall.java |   29 +-
 .../java/org/apache/calcite/sql/SqlJoin.java    |   12 +-
 .../java/org/apache/calcite/sql/SqlKind.java    |    6 +-
 .../java/org/apache/calcite/sql/SqlLiteral.java |   41 +-
 .../java/org/apache/calcite/sql/SqlMerge.java   |   18 +-
 .../java/org/apache/calcite/sql/SqlNode.java    |   43 +-
 .../org/apache/calcite/sql/SqlNodeList.java     |   20 +-
 .../apache/calcite/sql/SqlNullSemantics.java    |    2 +-
 .../apache/calcite/sql/SqlNumericLiteral.java   |   13 +-
 .../calcite/sql/SqlOperandCountRange.java       |    2 +-
 .../org/apache/calcite/sql/SqlOperator.java     |   53 +-
 .../apache/calcite/sql/SqlOperatorBinding.java  |   20 +-
 .../apache/calcite/sql/SqlOperatorTable.java    |    4 +-
 .../java/org/apache/calcite/sql/SqlOrderBy.java |   15 +-
 .../org/apache/calcite/sql/SqlOverOperator.java |   29 +-
 .../apache/calcite/sql/SqlPostfixOperator.java  |   16 +-
 .../apache/calcite/sql/SqlPrefixOperator.java   |   18 +-
 .../calcite/sql/SqlProcedureCallOperator.java   |   10 +-
 .../org/apache/calcite/sql/SqlRankFunction.java |   19 +-
 .../org/apache/calcite/sql/SqlSampleSpec.java   |   10 +-
 .../java/org/apache/calcite/sql/SqlSelect.java  |   14 +-
 .../apache/calcite/sql/SqlSelectKeyword.java    |    4 +-
 .../apache/calcite/sql/SqlSelectOperator.java   |   15 +-
 .../org/apache/calcite/sql/SqlSetOperator.java  |   11 +-
 .../org/apache/calcite/sql/SqlSetOption.java    |   25 +-
 .../apache/calcite/sql/SqlSpecialOperator.java  |   20 +-
 .../org/apache/calcite/sql/SqlStateCodes.java   |    7 +-
 .../java/org/apache/calcite/sql/SqlSyntax.java  |    7 +-
 .../org/apache/calcite/sql/SqlTimeLiteral.java  |   10 +-
 .../apache/calcite/sql/SqlTimestampLiteral.java |   10 +-
 .../apache/calcite/sql/SqlUnnestOperator.java   |    8 +-
 .../calcite/sql/SqlUnresolvedFunction.java      |   17 +-
 .../java/org/apache/calcite/sql/SqlUpdate.java  |   17 +-
 .../java/org/apache/calcite/sql/SqlUtil.java    |   64 +-
 .../apache/calcite/sql/SqlValuesOperator.java   |    2 +-
 .../java/org/apache/calcite/sql/SqlWindow.java  |   46 +-
 .../java/org/apache/calcite/sql/SqlWith.java    |   26 +-
 .../org/apache/calcite/sql/SqlWithItem.java     |   18 +-
 .../java/org/apache/calcite/sql/SqlWriter.java  |   13 +-
 .../apache/calcite/sql/advise/SqlAdvisor.java   |   45 +-
 .../sql/advise/SqlAdvisorGetHintsFunction.java  |   43 +-
 .../calcite/sql/advise/SqlAdvisorHint.java      |   14 +-
 .../calcite/sql/advise/SqlAdvisorValidator.java |   44 +-
 .../calcite/sql/advise/SqlSimpleParser.java     |   11 +-
 .../apache/calcite/sql/advise/package-info.java |    4 +-
 .../sql/fun/SqlAbstractTimeFunction.java        |   21 +-
 .../sql/fun/SqlArrayQueryConstructor.java       |    5 +-
 .../sql/fun/SqlArrayValueConstructor.java       |   13 +-
 .../calcite/sql/fun/SqlAvgAggFunction.java      |   17 +-
 .../calcite/sql/fun/SqlBetweenOperator.java     |   43 +-
 .../org/apache/calcite/sql/fun/SqlCase.java     |   15 +-
 .../apache/calcite/sql/fun/SqlCaseOperator.java |   38 +-
 .../apache/calcite/sql/fun/SqlCastFunction.java |   43 +-
 .../apache/calcite/sql/fun/SqlCeilFunction.java |   13 +-
 .../calcite/sql/fun/SqlCoalesceFunction.java    |   20 +-
 .../sql/fun/SqlCollectionTableOperator.java     |   12 +-
 .../sql/fun/SqlColumnListConstructor.java       |   11 +-
 .../calcite/sql/fun/SqlConvertFunction.java     |    8 +-
 .../calcite/sql/fun/SqlCountAggFunction.java    |   22 +-
 .../calcite/sql/fun/SqlCovarAggFunction.java    |   20 +-
 .../calcite/sql/fun/SqlCurrentDateFunction.java |   16 +-
 .../calcite/sql/fun/SqlCursorConstructor.java   |   16 +-
 .../sql/fun/SqlDatetimeSubtractionOperator.java |   16 +-
 .../calcite/sql/fun/SqlExtractFunction.java     |   13 +-
 .../sql/fun/SqlFirstLastValueAggFunction.java   |   17 +-
 .../calcite/sql/fun/SqlFloorFunction.java       |   13 +-
 .../sql/fun/SqlHistogramAggFunction.java        |   16 +-
 .../apache/calcite/sql/fun/SqlInOperator.java   |   33 +-
 .../apache/calcite/sql/fun/SqlItemOperator.java |   43 +-
 .../calcite/sql/fun/SqlLeadLagAggFunction.java  |   34 +-
 .../apache/calcite/sql/fun/SqlLikeOperator.java |   34 +-
 .../sql/fun/SqlLiteralChainOperator.java        |   49 +-
 .../calcite/sql/fun/SqlMapQueryConstructor.java |    4 +-
 .../calcite/sql/fun/SqlMapValueConstructor.java |   23 +-
 .../calcite/sql/fun/SqlMinMaxAggFunction.java   |   16 +-
 .../sql/fun/SqlMonotonicBinaryOperator.java     |   16 +-
 .../sql/fun/SqlMonotonicUnaryFunction.java      |   14 +-
 .../sql/fun/SqlMultisetMemberOfOperator.java    |   16 +-
 .../sql/fun/SqlMultisetQueryConstructor.java    |   25 +-
 .../calcite/sql/fun/SqlMultisetSetOperator.java |   13 +-
 .../sql/fun/SqlMultisetValueConstructor.java    |   21 +-
 .../apache/calcite/sql/fun/SqlNewOperator.java  |    9 +-
 .../calcite/sql/fun/SqlNtileAggFunction.java    |   18 +-
 .../calcite/sql/fun/SqlNullifFunction.java      |   19 +-
 .../calcite/sql/fun/SqlOverlapsOperator.java    |   23 +-
 .../calcite/sql/fun/SqlOverlayFunction.java     |   12 +-
 .../calcite/sql/fun/SqlPositionFunction.java    |   12 +-
 .../apache/calcite/sql/fun/SqlRowOperator.java  |   19 +-
 .../sql/fun/SqlSingleValueAggFunction.java      |   16 +-
 .../calcite/sql/fun/SqlStdOperatorTable.java    |   50 +-
 .../sql/fun/SqlStringContextVariable.java       |   14 +-
 .../calcite/sql/fun/SqlSubstringFunction.java   |   36 +-
 .../calcite/sql/fun/SqlSumAggFunction.java      |   16 +-
 .../sql/fun/SqlSumEmptyIsZeroAggFunction.java   |   17 +-
 .../calcite/sql/fun/SqlThrowOperator.java       |   10 +-
 .../apache/calcite/sql/fun/SqlTrimFunction.java |   30 +-
 .../apache/calcite/sql/fun/package-info.java    |    6 +-
 .../org/apache/calcite/sql/package-info.java    |   96 +-
 .../sql/parser/SqlAbstractParserImpl.java       |   34 +-
 .../calcite/sql/parser/SqlParseException.java   |   19 +-
 .../apache/calcite/sql/parser/SqlParser.java    |   31 +-
 .../sql/parser/SqlParserImplFactory.java        |   11 +-
 .../apache/calcite/sql/parser/SqlParserPos.java |   11 +-
 .../calcite/sql/parser/SqlParserUtil.java       |   53 +-
 .../calcite/sql/parser/impl/package-info.java   |    4 +-
 .../apache/calcite/sql/parser/package-info.java |    2 +-
 .../calcite/sql/pretty/SqlFormatOptions.java    |    2 +-
 .../calcite/sql/pretty/SqlPrettyWriter.java     |   60 +-
 .../apache/calcite/sql/pretty/package-info.java |    2 +-
 .../calcite/sql/type/AbstractSqlType.java       |   12 +-
 .../apache/calcite/sql/type/ArraySqlType.java   |    6 +-
 .../sql/type/AssignableOperandTypeChecker.java  |   18 +-
 .../apache/calcite/sql/type/BasicSqlType.java   |   11 +-
 .../sql/type/ComparableOperandTypeChecker.java  |   17 +-
 .../sql/type/CompositeOperandTypeChecker.java   |   18 +-
 .../sql/type/CursorReturnTypeInference.java     |    6 +-
 .../sql/type/ExplicitOperandTypeInference.java  |    9 +-
 .../sql/type/ExplicitReturnTypeInference.java   |    8 +-
 .../apache/calcite/sql/type/ExtraSqlTypes.java  |    2 +-
 .../sql/type/FamilyOperandTypeChecker.java      |   19 +-
 .../org/apache/calcite/sql/type/InferTypes.java |   18 +-
 .../calcite/sql/type/IntervalSqlType.java       |   18 +-
 .../sql/type/JavaToSqlTypeConversionRules.java  |   20 +-
 .../sql/type/LiteralOperandTypeChecker.java     |   12 +-
 .../org/apache/calcite/sql/type/MapSqlType.java |   12 +-
 .../sql/type/MatchReturnTypeInference.java      |   10 +-
 .../sql/type/MultisetOperandTypeChecker.java    |   12 +-
 .../calcite/sql/type/MultisetSqlType.java       |    6 +-
 .../apache/calcite/sql/type/ObjectSqlType.java  |   10 +-
 .../apache/calcite/sql/type/OperandTypes.java   |   28 +-
 .../calcite/sql/type/OperandsTypeChecking.java  |    5 +-
 .../sql/type/OrdinalReturnTypeInference.java    |    7 +-
 .../apache/calcite/sql/type/ReturnTypes.java    |   45 +-
 .../sql/type/SameOperandTypeChecker.java        |   25 +-
 .../sql/type/SetopOperandTypeChecker.java       |   21 +-
 .../calcite/sql/type/SqlOperandCountRanges.java |    5 +-
 .../calcite/sql/type/SqlOperandTypeChecker.java |   10 +-
 .../sql/type/SqlOperandTypeInference.java       |    6 +-
 .../sql/type/SqlReturnTypeInference.java        |   12 +-
 .../sql/type/SqlReturnTypeInferenceChain.java   |    8 +-
 .../sql/type/SqlSingleOperandTypeChecker.java   |    5 +-
 .../sql/type/SqlTypeAssignmentRules.java        |    9 +-
 .../sql/type/SqlTypeExplicitPrecedenceList.java |   14 +-
 .../calcite/sql/type/SqlTypeFactoryImpl.java    |   19 +-
 .../apache/calcite/sql/type/SqlTypeFamily.java  |   12 +-
 .../apache/calcite/sql/type/SqlTypeName.java    |   32 +-
 .../calcite/sql/type/SqlTypeTransform.java      |   10 +-
 .../sql/type/SqlTypeTransformCascade.java       |   10 +-
 .../calcite/sql/type/SqlTypeTransforms.java     |   17 +-
 .../apache/calcite/sql/type/SqlTypeUtil.java    |   58 +-
 .../type/TableFunctionReturnTypeInference.java  |   18 +-
 .../apache/calcite/sql/type/package-info.java   |    2 +-
 .../sql/util/ChainedSqlOperatorTable.java       |   11 +-
 .../calcite/sql/util/ListSqlOperatorTable.java  |   12 +-
 .../sql/util/ReflectiveSqlOperatorTable.java    |   32 +-
 .../calcite/sql/util/SqlBasicVisitor.java       |   29 +-
 .../org/apache/calcite/sql/util/SqlBuilder.java |   13 +-
 .../org/apache/calcite/sql/util/SqlShuttle.java |   18 +-
 .../org/apache/calcite/sql/util/SqlString.java  |   13 +-
 .../org/apache/calcite/sql/util/SqlVisitor.java |   18 +-
 .../apache/calcite/sql/util/package-info.java   |    2 +-
 .../calcite/sql/validate/AbstractNamespace.java |   18 +-
 .../apache/calcite/sql/validate/AggChecker.java |   15 +-
 .../apache/calcite/sql/validate/AggFinder.java  |   23 +-
 .../calcite/sql/validate/AggregatingScope.java  |    4 +-
 .../sql/validate/AggregatingSelectScope.java    |   14 +-
 .../calcite/sql/validate/AliasNamespace.java    |   21 +-
 .../calcite/sql/validate/CollectNamespace.java  |    8 +-
 .../calcite/sql/validate/CollectScope.java      |    5 +-
 .../sql/validate/DelegatingNamespace.java       |   10 +-
 .../calcite/sql/validate/DelegatingScope.java   |   26 +-
 .../DelegatingSqlValidatorCatalogReader.java    |   10 +-
 .../validate/DelegatingSqlValidatorTable.java   |   10 +-
 .../apache/calcite/sql/validate/EmptyScope.java |   17 +-
 .../calcite/sql/validate/FieldNamespace.java    |    6 +-
 .../sql/validate/IdentifierNamespace.java       |   24 +-
 .../calcite/sql/validate/JoinNamespace.java     |    8 +-
 .../apache/calcite/sql/validate/JoinScope.java  |    6 +-
 .../apache/calcite/sql/validate/ListScope.java  |   15 +-
 .../calcite/sql/validate/OrderByScope.java      |   12 +-
 .../apache/calcite/sql/validate/OverScope.java  |    9 +-
 .../sql/validate/ParameterNamespace.java        |    6 +-
 .../calcite/sql/validate/ParameterScope.java    |   13 +-
 .../sql/validate/ProcedureNamespace.java        |   10 +-
 .../calcite/sql/validate/SelectNamespace.java   |    9 +-
 .../calcite/sql/validate/SelectScope.java       |   22 +-
 .../calcite/sql/validate/SetopNamespace.java    |   12 +-
 .../calcite/sql/validate/SqlConformance.java    |    2 +-
 .../sql/validate/SqlIdentifierMoniker.java      |    8 +-
 .../apache/calcite/sql/validate/SqlMoniker.java |    6 +-
 .../sql/validate/SqlMonikerComparator.java      |    8 +-
 .../calcite/sql/validate/SqlMonikerImpl.java    |   12 +-
 .../calcite/sql/validate/SqlMonikerType.java    |    2 +-
 .../calcite/sql/validate/SqlMonotonicity.java   |    2 +-
 .../calcite/sql/validate/SqlScopedShuttle.java  |   10 +-
 .../sql/validate/SqlUserDefinedAggFunction.java |   25 +-
 .../sql/validate/SqlUserDefinedFunction.java    |   20 +-
 .../validate/SqlUserDefinedTableFunction.java   |   25 +-
 .../sql/validate/SqlUserDefinedTableMacro.java  |   65 +-
 .../calcite/sql/validate/SqlValidator.java      |  106 +-
 .../sql/validate/SqlValidatorCatalogReader.java |    9 +-
 .../sql/validate/SqlValidatorException.java     |   20 +-
 .../calcite/sql/validate/SqlValidatorImpl.java  |  138 +-
 .../sql/validate/SqlValidatorNamespace.java     |   36 +-
 .../calcite/sql/validate/SqlValidatorScope.java |   31 +-
 .../calcite/sql/validate/SqlValidatorTable.java |    8 +-
 .../calcite/sql/validate/SqlValidatorUtil.java  |   54 +-
 .../sql/validate/SqlValidatorWithHints.java     |   16 +-
 .../sql/validate/TableConstructorNamespace.java |   10 +-
 .../calcite/sql/validate/TableNamespace.java    |    9 +-
 .../calcite/sql/validate/UnnestNamespace.java   |   11 +-
 .../calcite/sql/validate/WithItemNamespace.java |   19 +-
 .../calcite/sql/validate/WithNamespace.java     |   10 +-
 .../apache/calcite/sql/validate/WithScope.java  |   14 +-
 .../calcite/sql/validate/package-info.java      |    2 +-
 .../calcite/sql2rel/DefaultValueFactory.java    |   12 +-
 .../sql2rel/ReflectiveConvertletTable.java      |   24 +-
 .../apache/calcite/sql2rel/RelDecorrelator.java |  576 +--
 .../apache/calcite/sql2rel/RelFieldTrimmer.java |  139 +-
 .../sql2rel/RelStructuredTypeFlattener.java     |  167 +-
 .../calcite/sql2rel/SqlNodeToRexConverter.java  |   14 +-
 .../sql2rel/SqlNodeToRexConverterImpl.java      |   37 +-
 .../apache/calcite/sql2rel/SqlRexContext.java   |   21 +-
 .../calcite/sql2rel/SqlRexConvertlet.java       |    7 +-
 .../calcite/sql2rel/SqlRexConvertletTable.java  |    4 +-
 .../calcite/sql2rel/SqlToRelConverter.java      |  405 ++-
 .../sql2rel/StandardConvertletTable.java        |   83 +-
 .../calcite/sql2rel/SubqueryConverter.java      |    6 +-
 .../apache/calcite/sql2rel/package-info.java    |    2 +-
 .../apache/calcite/tools/FrameworkConfig.java   |   25 +-
 .../org/apache/calcite/tools/Frameworks.java    |   67 +-
 .../java/org/apache/calcite/tools/Planner.java  |   20 +-
 .../java/org/apache/calcite/tools/Program.java  |   10 +-
 .../java/org/apache/calcite/tools/Programs.java |  157 +-
 .../calcite/tools/RelConversionException.java   |    4 +-
 .../java/org/apache/calcite/tools/RuleSet.java  |    4 +-
 .../java/org/apache/calcite/tools/RuleSets.java |   12 +-
 .../calcite/tools/ValidationException.java      |    2 +-
 .../org/apache/calcite/tools/package-info.java  |    2 +-
 .../calcite/util/BarfingInvocationHandler.java  |    6 +-
 .../org/apache/calcite/util/BasicDatetime.java  |   10 +-
 .../java/org/apache/calcite/util/Benchmark.java |   10 +-
 .../java/org/apache/calcite/util/BitSets.java   |   12 +-
 .../java/org/apache/calcite/util/BitString.java |   19 +-
 .../main/java/org/apache/calcite/util/Bug.java  |    4 +-
 .../org/apache/calcite/util/BuiltInMethod.java  |   94 +-
 .../calcite/util/CalciteParserException.java    |   11 +-
 .../calcite/util/CalciteValidatorException.java |   11 +-
 .../org/apache/calcite/util/CancelFlag.java     |    2 +-
 .../org/apache/calcite/util/CastingList.java    |   11 +-
 .../java/org/apache/calcite/util/ChunkList.java |   23 +-
 .../apache/calcite/util/ClosableAllocation.java |    2 +-
 .../calcite/util/ClosableAllocationOwner.java   |    2 +-
 .../org/apache/calcite/util/Compatible.java     |   15 +-
 .../apache/calcite/util/CompatibleGuava11.java  |   80 +-
 .../org/apache/calcite/util/CompositeList.java  |    7 +-
 .../org/apache/calcite/util/CompositeMap.java   |    7 +-
 .../util/CompoundClosableAllocation.java        |    6 +-
 .../calcite/util/ControlFlowException.java      |    2 +-
 .../org/apache/calcite/util/ConversionUtil.java |   12 +-
 .../org/apache/calcite/util/DateTimeUtil.java   |   10 +-
 .../util/DelegatingInvocationHandler.java       |    6 +-
 .../org/apache/calcite/util/Filterator.java     |   13 +-
 .../java/org/apache/calcite/util/Glossary.java  |    2 +-
 .../java/org/apache/calcite/util/Holder.java    |    2 +-
 .../apache/calcite/util/ImmutableIntList.java   |   43 +-
 .../calcite/util/ImmutableNullableList.java     |    9 +-
 .../java/org/apache/calcite/util/IntList.java   |    7 +-
 .../apache/calcite/util/IntegerIntervalSet.java |   18 +-
 .../org/apache/calcite/util/JsonBuilder.java    |    6 +-
 .../java/org/apache/calcite/util/NlsString.java |   30 +-
 .../org/apache/calcite/util/NumberUtil.java     |    9 +-
 .../main/java/org/apache/calcite/util/Pair.java |   27 +-
 .../calcite/util/PartiallyOrderedSet.java       |   47 +-
 .../org/apache/calcite/util/Permutation.java    |   19 +-
 .../org/apache/calcite/util/ReflectUtil.java    |   37 +-
 .../calcite/util/ReflectiveVisitDispatcher.java |   14 +-
 .../apache/calcite/util/ReflectiveVisitor.java  |    6 +-
 .../java/org/apache/calcite/util/RhBase64.java  |   23 +-
 .../apache/calcite/util/SaffronProperties.java  |   42 +-
 .../calcite/util/SerializableCharset.java       |   14 +-
 .../org/apache/calcite/util/StackWriter.java    |   15 +-
 .../java/org/apache/calcite/util/Stacks.java    |    2 +-
 .../java/org/apache/calcite/util/Static.java    |   34 +-
 .../java/org/apache/calcite/util/Template.java  |   11 +-
 .../calcite/util/UnmodifiableArrayList.java     |    6 +-
 .../main/java/org/apache/calcite/util/Util.java |  226 +-
 .../java/org/apache/calcite/util/XmlOutput.java |   14 +-
 .../org/apache/calcite/util/ZonelessDate.java   |   13 +-
 .../apache/calcite/util/ZonelessDatetime.java   |   23 +-
 .../org/apache/calcite/util/ZonelessTime.java   |    7 +-
 .../apache/calcite/util/ZonelessTimestamp.java  |    9 +-
 .../util/graph/BreadthFirstIterator.java        |    2 +-
 .../calcite/util/graph/CycleDetector.java       |    3 +-
 .../util/graph/DefaultDirectedGraph.java        |   17 +-
 .../apache/calcite/util/graph/DefaultEdge.java  |    8 +-
 .../calcite/util/graph/DepthFirstIterator.java  |    8 +-
 .../calcite/util/graph/DirectedGraph.java       |    2 +-
 .../org/apache/calcite/util/graph/Graphs.java   |   12 +-
 .../util/graph/TopologicalOrderIterator.java    |    9 +-
 .../apache/calcite/util/graph/package-info.java |    2 +-
 .../calcite/util/javac/JaninoCompiler.java      |   22 +-
 .../apache/calcite/util/javac/JavaCompiler.java |    2 +-
 .../calcite/util/javac/JavaCompilerArgs.java    |   14 +-
 .../apache/calcite/util/javac/package-info.java |    2 +-
 .../util/mapping/AbstractSourceMapping.java     |    8 +-
 .../util/mapping/AbstractTargetMapping.java     |    6 +-
 .../apache/calcite/util/mapping/IntPair.java    |    2 +-
 .../apache/calcite/util/mapping/Mapping.java    |    8 +-
 .../calcite/util/mapping/MappingType.java       |   22 +-
 .../apache/calcite/util/mapping/Mappings.java   |   53 +-
 .../calcite/util/mapping/package-info.java      |    2 +-
 .../org/apache/calcite/util/package-info.java   |    2 +-
 .../calcite/util/trace/CalciteLogger.java       |   46 +-
 .../calcite/util/trace/CalciteTimingTracer.java |   16 +-
 .../apache/calcite/util/trace/CalciteTrace.java |   76 +-
 .../apache/calcite/util/trace/package-info.java |    2 +-
 .../resources/META-INF/services/java.sql.Driver |    2 +-
 .../calcite/runtime/CalciteResource.properties  |    6 +-
 .../calcite/adapter/clone/ArrayTableTest.java   |   21 +-
 .../calcite/adapter/generate/RangeTable.java    |   32 +-
 .../examples/foodmart/java/JdbcExample.java     |   32 +-
 .../calcite/plan/RelOptPlanReaderTest.java      |   44 +-
 .../org/apache/calcite/plan/RelOptUtilTest.java |   20 +-
 .../org/apache/calcite/plan/RelWriterTest.java  |   76 +-
 .../plan/volcano/VolcanoPlannerTest.java        |   87 +-
 .../plan/volcano/VolcanoPlannerTraitTest.java   |   77 +-
 .../org/apache/calcite/rex/RexExecutorTest.java |   47 +-
 .../calcite/runtime/BinarySearchTest.java       |    4 +-
 .../apache/calcite/runtime/EnumerablesTest.java |   10 +-
 .../calcite/sql/parser/SqlParserTest.java       | 1140 +++---
 .../calcite/sql/parser/SqlUnParserTest.java     |    2 +-
 .../calcite/sql/test/DefaultSqlTestFactory.java |   32 +-
 .../sql/test/DelegatingSqlTestFactory.java      |   23 +-
 .../apache/calcite/sql/test/SqlAdvisorTest.java |  271 +-
 .../calcite/sql/test/SqlOperatorBaseTest.java   |  169 +-
 .../calcite/sql/test/SqlOperatorTest.java       |    6 +-
 .../calcite/sql/test/SqlPrettyWriterTest.java   |   75 +-
 .../apache/calcite/sql/test/SqlTestFactory.java |   14 +-
 .../org/apache/calcite/sql/test/SqlTester.java  |   22 +-
 .../apache/calcite/sql/test/SqlTesterImpl.java  |   79 +-
 .../org/apache/calcite/sql/test/SqlTests.java   |   33 +-
 .../calcite/sql/test/SqlTypeNameTest.java       |   11 +-
 .../apache/calcite/sql/test/package-info.java   |    2 +-
 .../org/apache/calcite/test/CalciteAssert.java  |  327 +-
 .../calcite/test/CalciteResourceTest.java       |   21 +-
 .../calcite/test/CalciteSqlOperatorTest.java    |   25 +-
 .../org/apache/calcite/test/CalciteSuite.java   |   53 +-
 .../org/apache/calcite/test/DiffRepository.java |   51 +-
 .../org/apache/calcite/test/DiffTestCase.java   |   37 +-
 .../calcite/test/ExceptionMessageTest.java      |   23 +-
 .../org/apache/calcite/test/FilteratorTest.java |   18 +-
 .../org/apache/calcite/test/FoodmartTest.java   |   39 +-
 .../org/apache/calcite/test/HepPlannerTest.java |   46 +-
 .../apache/calcite/test/InterpreterTest.java    |   38 +-
 .../apache/calcite/test/JdbcAdapterTest.java    |   39 +-
 .../test/JdbcFrontJdbcBackLinqMiddleTest.java   |  121 +-
 .../calcite/test/JdbcFrontJdbcBackTest.java     |   49 +-
 .../calcite/test/JdbcFrontLinqBackTest.java     |  120 +-
 .../java/org/apache/calcite/test/JdbcTest.java  | 2969 ++++++++--------
 .../org/apache/calcite/test/LatticeTest.java    |  128 +-
 .../calcite/test/LinqFrontJdbcBackTest.java     |   22 +-
 .../calcite/test/MaterializationTest.java       |  112 +-
 .../apache/calcite/test/MockCatalogReader.java  |   61 +-
 .../org/apache/calcite/test/MockRelOptCost.java |   10 +-
 .../apache/calcite/test/MockRelOptPlanner.java  |   32 +-
 .../calcite/test/MockSqlOperatorTable.java      |   26 +-
 .../java/org/apache/calcite/test/ModelTest.java |  176 +-
 .../calcite/test/MultiJdbcSchemaJoinTest.java   |   80 +-
 .../calcite/test/ReflectiveSchemaTest.java      |  222 +-
 .../apache/calcite/test/RelMetadataTest.java    |  108 +-
 .../apache/calcite/test/RelOptRulesTest.java    |  529 ++-
 .../org/apache/calcite/test/RelOptTestBase.java |   36 +-
 .../org/apache/calcite/test/RexProgramTest.java |  135 +-
 .../apache/calcite/test/RexTransformerTest.java |   22 +-
 .../apache/calcite/test/ScannableTableTest.java |  117 +-
 .../apache/calcite/test/SqlFunctionsTest.java   |   49 +-
 .../org/apache/calcite/test/SqlLimitsTest.java  |   33 +-
 .../org/apache/calcite/test/SqlTestGen.java     |   54 +-
 .../test/SqlToRelConverterExtendedTest.java     |   25 +-
 .../calcite/test/SqlToRelConverterTest.java     |  270 +-
 .../apache/calcite/test/SqlToRelTestBase.java   |   83 +-
 .../calcite/test/SqlValidatorFeatureTest.java   |   45 +-
 .../apache/calcite/test/SqlValidatorTest.java   |  969 +++--
 .../calcite/test/SqlValidatorTestCase.java      |   56 +-
 .../calcite/test/TableInRootSchemaTest.java     |   74 +-
 .../test/concurrent/ConcurrentTestCommand.java  |    2 +-
 .../ConcurrentTestCommandExecutor.java          |   15 +-
 .../ConcurrentTestCommandGenerator.java         |   83 +-
 .../concurrent/ConcurrentTestCommandScript.java |  116 +-
 .../test/concurrent/ConcurrentTestPlugin.java   |    2 +-
 .../concurrent/ConcurrentTestPluginCommand.java |    3 +-
 .../ConcurrentTestTimedCommandGenerator.java    |   19 +-
 .../calcite/test/concurrent/SamplePlugin.java   |   12 +-
 .../calcite/test/concurrent/package-info.java   |   14 +-
 .../org/apache/calcite/test/package-info.java   |    4 +-
 .../apache/calcite/tools/FrameworksTest.java    |   73 +-
 .../org/apache/calcite/tools/PlannerTest.java   |  491 +--
 .../org/apache/calcite/tools/TpchSchema.java    |    8 +-
 .../org/apache/calcite/util/BitSetsTest.java    |   24 +-
 .../org/apache/calcite/util/ChunkListTest.java  |   28 +-
 .../calcite/util/PartiallyOrderedSetTest.java   |   39 +-
 .../calcite/util/PermutationTestCase.java       |    7 +-
 .../apache/calcite/util/ReflectVisitorTest.java |   20 +-
 .../java/org/apache/calcite/util/TestUtil.java  |   10 +-
 .../java/org/apache/calcite/util/UtilTest.java  |  131 +-
 .../calcite/util/graph/DirectedGraphTest.java   |   23 +-
 .../calcite/util/mapping/MappingTest.java       |   18 +-
 .../org/apache/calcite/test/HepPlannerTest.xml  |  192 +-
 .../org/apache/calcite/test/RelOptRulesTest.xml | 2216 ++++++------
 .../calcite/test/SqlToRelConverterTest.xml      | 1190 +++----
 core/src/test/resources/sql/misc.oq             |  166 +-
 core/src/test/resources/sql/subquery.oq         |   50 +-
 .../calcite/adapter/csv/CsvEnumerator.java      |   21 +-
 .../calcite/adapter/csv/CsvFieldType.java       |   10 +-
 .../calcite/adapter/csv/CsvFilterableTable.java |   20 +-
 .../adapter/csv/CsvProjectTableScanRule.java    |   31 +-
 .../calcite/adapter/csv/CsvScannableTable.java  |   16 +-
 .../apache/calcite/adapter/csv/CsvSchema.java   |   14 +-
 .../calcite/adapter/csv/CsvSchemaFactory.java   |    8 +-
 .../apache/calcite/adapter/csv/CsvTable.java    |   11 +-
 .../calcite/adapter/csv/CsvTableFactory.java    |   12 +-
 .../calcite/adapter/csv/CsvTableScan.java       |   45 +-
 .../adapter/csv/CsvTranslatableTable.java       |   30 +-
 .../calcite/adapter/csv/JsonEnumerator.java     |    9 +-
 .../apache/calcite/adapter/csv/JsonTable.java   |   24 +-
 .../calcite/adapter/csv/package-info.java       |    2 +-
 .../java/org/apache/calcite/test/CsvTest.java   |   50 +-
 .../org/apache/calcite/test/package-info.java   |   23 -
 example/csv/src/test/resources/bug.json         |    2 +-
 .../src/test/resources/filterable-model.json    |    2 +-
 .../test/resources/model-with-custom-table.json |    2 +-
 .../csv/src/test/resources/model-with-view.json |    2 +-
 example/csv/src/test/resources/model.json       |    2 +-
 example/csv/src/test/resources/smart.json       |    2 +-
 linq4j/pom.xml                                  |    2 +-
 .../calcite/linq4j/AbstractEnumerable.java      |    2 +-
 .../calcite/linq4j/AbstractEnumerable2.java     |    6 +-
 .../calcite/linq4j/AbstractQueryable.java       |    2 +-
 .../apache/calcite/linq4j/BaseQueryable.java    |    4 +-
 .../linq4j/CartesianProductEnumerator.java      |    2 +-
 .../calcite/linq4j/DefaultEnumerable.java       |   30 +-
 .../apache/calcite/linq4j/DefaultQueryable.java |   73 +-
 .../org/apache/calcite/linq4j/Enumerable.java   |    2 +-
 .../calcite/linq4j/EnumerableDefaults.java      |   78 +-
 .../linq4j/EnumerableOrderedQueryable.java      |   10 +-
 .../calcite/linq4j/EnumerableQueryable.java     |   51 +-
 .../org/apache/calcite/linq4j/Enumerator.java   |    2 +-
 .../calcite/linq4j/ExtendedEnumerable.java      |   26 +-
 .../linq4j/ExtendedOrderedEnumerable.java       |    9 +-
 .../linq4j/ExtendedOrderedQueryable.java        |    6 +-
 .../calcite/linq4j/ExtendedQueryable.java       |   32 +-
 .../org/apache/calcite/linq4j/Extensions.java   |   18 +-
 .../org/apache/calcite/linq4j/Grouping.java     |    2 +-
 .../org/apache/calcite/linq4j/GroupingImpl.java |   11 +-
 .../java/org/apache/calcite/linq4j/Linq4j.java  |   40 +-
 .../java/org/apache/calcite/linq4j/Lookup.java  |    4 +-
 .../org/apache/calcite/linq4j/LookupImpl.java   |   17 +-
 .../java/org/apache/calcite/linq4j/OpType.java  |    2 +-
 .../java/org/apache/calcite/linq4j/Ord.java     |    8 +-
 .../calcite/linq4j/OrderedEnumerable.java       |    4 +-
 .../apache/calcite/linq4j/OrderedQueryable.java |    2 +-
 .../apache/calcite/linq4j/QueryProvider.java    |    8 +-
 .../calcite/linq4j/QueryProviderImpl.java       |    7 +-
 .../org/apache/calcite/linq4j/Queryable.java    |    2 +-
 .../calcite/linq4j/QueryableDefaults.java       |   49 +-
 .../apache/calcite/linq4j/QueryableFactory.java |   22 +-
 .../calcite/linq4j/QueryableRecorder.java       |   29 +-
 .../apache/calcite/linq4j/RawEnumerable.java    |    2 +-
 .../org/apache/calcite/linq4j/RawQueryable.java |    4 +-
 .../linq4j/function/BigDecimalFunction1.java    |    2 +-
 .../calcite/linq4j/function/Deterministic.java  |    2 +-
 .../linq4j/function/DoubleFunction1.java        |    2 +-
 .../linq4j/function/EqualityComparer.java       |    2 +-
 .../calcite/linq4j/function/FloatFunction1.java |    2 +-
 .../calcite/linq4j/function/Function.java       |    2 +-
 .../calcite/linq4j/function/Function0.java      |    2 +-
 .../calcite/linq4j/function/Function1.java      |    2 +-
 .../calcite/linq4j/function/Function2.java      |    2 +-
 .../calcite/linq4j/function/Functions.java      |   13 +-
 .../linq4j/function/IntegerFunction1.java       |    2 +-
 .../calcite/linq4j/function/LongFunction1.java  |    2 +-
 .../linq4j/function/NonDeterministic.java       |    2 +-
 .../function/NullableBigDecimalFunction1.java   |    2 +-
 .../function/NullableDoubleFunction1.java       |    2 +-
 .../linq4j/function/NullableFloatFunction1.java |    2 +-
 .../function/NullableIntegerFunction1.java      |    2 +-
 .../linq4j/function/NullableLongFunction1.java  |    2 +-
 .../calcite/linq4j/function/Predicate1.java     |    2 +-
 .../calcite/linq4j/function/Predicate2.java     |    6 +-
 .../calcite/linq4j/function/package-info.java   |    2 +-
 .../org/apache/calcite/linq4j/package-info.java |    2 +-
 .../calcite/linq4j/tree/AbstractNode.java       |   11 +-
 .../linq4j/tree/ArrayLengthRecordField.java     |   10 +-
 .../calcite/linq4j/tree/BinaryExpression.java   |   19 +-
 .../calcite/linq4j/tree/BlockBuilder.java       |   21 +-
 .../calcite/linq4j/tree/BlockStatement.java     |   17 +-
 .../org/apache/calcite/linq4j/tree/Blocks.java  |    8 +-
 .../calcite/linq4j/tree/CallSiteBinder.java     |    2 +-
 .../apache/calcite/linq4j/tree/CatchBlock.java  |    8 +-
 .../calcite/linq4j/tree/ClassDeclaration.java   |    8 +-
 .../linq4j/tree/ClassDeclarationFinder.java     |   16 +-
 .../linq4j/tree/ConditionalExpression.java      |   11 +-
 .../linq4j/tree/ConditionalStatement.java       |   14 +-
 .../calcite/linq4j/tree/ConstantExpression.java |   18 +-
 .../linq4j/tree/ConstantUntypedNull.java        |   13 +-
 .../linq4j/tree/ConstructorDeclaration.java     |   15 +-
 .../linq4j/tree/DeclarationStatement.java       |   14 +-
 .../calcite/linq4j/tree/DefaultExpression.java  |    5 +-
 .../linq4j/tree/DeterministicCodeOptimizer.java |   48 +-
 .../calcite/linq4j/tree/DynamicExpression.java  |    5 +-
 .../apache/calcite/linq4j/tree/ElementInit.java |    4 +-
 .../apache/calcite/linq4j/tree/Evaluator.java   |    2 +-
 .../apache/calcite/linq4j/tree/Expression.java  |    5 +-
 .../calcite/linq4j/tree/ExpressionType.java     |    2 +-
 .../calcite/linq4j/tree/ExpressionVisitor.java  |    4 +-
 .../calcite/linq4j/tree/ExpressionWriter.java   |    9 +-
 .../apache/calcite/linq4j/tree/Expressions.java |   28 +-
 .../calcite/linq4j/tree/FieldDeclaration.java   |   11 +-
 .../calcite/linq4j/tree/ForStatement.java       |   16 +-
 .../calcite/linq4j/tree/FunctionExpression.java |   33 +-
 .../calcite/linq4j/tree/GotoExpressionKind.java |    2 +-
 .../calcite/linq4j/tree/GotoStatement.java      |   17 +-
 .../calcite/linq4j/tree/IndexExpression.java    |   14 +-
 .../linq4j/tree/InvocationExpression.java       |    5 +-
 .../calcite/linq4j/tree/LabelStatement.java     |   11 +-
 .../apache/calcite/linq4j/tree/LabelTarget.java |    8 +-
 .../calcite/linq4j/tree/LambdaExpression.java   |    5 +-
 .../calcite/linq4j/tree/ListInitExpression.java |    5 +-
 .../calcite/linq4j/tree/MemberAssignment.java   |    2 +-
 .../calcite/linq4j/tree/MemberBinding.java      |    2 +-
 .../calcite/linq4j/tree/MemberDeclaration.java  |    2 +-
 .../calcite/linq4j/tree/MemberExpression.java   |   14 +-
 .../linq4j/tree/MemberInitExpression.java       |    5 +-
 .../calcite/linq4j/tree/MemberListBinding.java  |    2 +-
 .../linq4j/tree/MemberMemberBinding.java        |    2 +-
 .../linq4j/tree/MethodCallExpression.java       |   17 +-
 .../calcite/linq4j/tree/MethodDeclaration.java  |   15 +-
 .../calcite/linq4j/tree/NewArrayExpression.java |   14 +-
 .../calcite/linq4j/tree/NewExpression.java      |   14 +-
 .../org/apache/calcite/linq4j/tree/Node.java    |    2 +-
 .../calcite/linq4j/tree/OptimizeVisitor.java    |   28 +-
 .../linq4j/tree/ParameterExpression.java        |   21 +-
 .../apache/calcite/linq4j/tree/Primitive.java   |    9 +-
 .../apache/calcite/linq4j/tree/PseudoField.java |    2 +-
 .../linq4j/tree/ReflectedPseudoField.java       |   10 +-
 .../apache/calcite/linq4j/tree/Statement.java   |    8 +-
 .../apache/calcite/linq4j/tree/SwitchCase.java  |    2 +-
 .../calcite/linq4j/tree/SwitchStatement.java    |    5 +-
 .../calcite/linq4j/tree/TernaryExpression.java  |   11 +-
 .../calcite/linq4j/tree/ThrowStatement.java     |   14 +-
 .../calcite/linq4j/tree/TryStatement.java       |   14 +-
 .../linq4j/tree/TypeBinaryExpression.java       |   11 +-
 .../org/apache/calcite/linq4j/tree/Types.java   |   37 +-
 .../calcite/linq4j/tree/UnaryExpression.java    |   11 +-
 .../org/apache/calcite/linq4j/tree/Visitor.java |    4 +-
 .../calcite/linq4j/tree/WhileStatement.java     |   14 +-
 .../calcite/linq4j/tree/package-info.java       |    4 +-
 .../test/java/com/example/Linq4jExample.java    |    7 +-
 .../calcite/linq4j/function/FunctionTest.java   |    6 +-
 .../calcite/linq4j/function/package-info.java   |    2 +-
 .../calcite/linq4j/test/BlockBuilderBase.java   |    9 +-
 .../calcite/linq4j/test/BlockBuilderTest.java   |   24 +-
 .../calcite/linq4j/test/DeterministicTest.java  |   89 +-
 .../calcite/linq4j/test/ExpressionTest.java     |  430 +--
 .../apache/calcite/linq4j/test/InlinerTest.java |   45 +-
 .../apache/calcite/linq4j/test/Linq4jSuite.java |    6 +-
 .../apache/calcite/linq4j/test/Linq4jTest.java  |  161 +-
 .../calcite/linq4j/test/OptimizerTest.java      |  106 +-
 .../calcite/linq4j/test/PrimitiveTest.java      |   38 +-
 .../calcite/linq4j/test/package-info.java       |    2 +-
 .../apache/calcite/linq4j/tree/TypeTest.java    |    4 +-
 .../calcite/linq4j/tree/package-info.java       |    2 +-
 mongodb/pom.xml                                 |   21 +-
 .../calcite/adapter/mongodb/MongoAggregate.java |   53 +-
 .../adapter/mongodb/MongoEnumerator.java        |   17 +-
 .../calcite/adapter/mongodb/MongoFilter.java    |   49 +-
 .../calcite/adapter/mongodb/MongoMethod.java    |    6 +-
 .../calcite/adapter/mongodb/MongoProject.java   |   42 +-
 .../calcite/adapter/mongodb/MongoRel.java       |   10 +-
 .../calcite/adapter/mongodb/MongoRules.java     |  161 +-
 .../calcite/adapter/mongodb/MongoSchema.java    |   12 +-
 .../adapter/mongodb/MongoSchemaFactory.java     |    6 +-
 .../calcite/adapter/mongodb/MongoSort.java      |   46 +-
 .../calcite/adapter/mongodb/MongoTable.java     |   56 +-
 .../calcite/adapter/mongodb/MongoTableScan.java |   30 +-
 .../mongodb/MongoToEnumerableConverter.java     |   49 +-
 .../mongodb/MongoToEnumerableConverterRule.java |   14 +-
 .../calcite/adapter/mongodb/package-info.java   |    2 +-
 .../apache/calcite/test/MongoAdapterTest.java   |  342 +-
 .../test/resources/mongo-foodmart-model.json    |    2 +-
 .../src/test/resources/mongo-zips-model.json    |    2 +-
 plus/pom.xml                                    |    2 +-
 .../calcite/adapter/tpcds/TpcdsSchema.java      |   46 +-
 .../adapter/tpcds/TpcdsSchemaFactory.java       |   11 +-
 .../calcite/adapter/tpcds/package-info.java     |    2 +-
 .../apache/calcite/adapter/tpch/TpchSchema.java |   40 +-
 .../calcite/adapter/tpch/TpchSchemaFactory.java |   11 +-
 .../calcite/adapter/tpch/package-info.java      |    2 +-
 .../apache/calcite/adapter/tpcds/TpcdsTest.java |  195 ++
 .../apache/calcite/adapter/tpch/TpcdsTest.java  |  197 --
 .../apache/calcite/adapter/tpch/TpchTest.java   |  939 +++++
 .../org/apache/calcite/impl/tpch/TpchTest.java  |  941 -----
 .../java/org/apache/calcite/test/PlusSuite.java |    6 +-
 pom.xml                                         |    2 +-
 spark/pom.xml                                   |   22 +-
 .../spark/EnumerableToSparkConverter.java       |   36 +-
 .../spark/EnumerableToSparkConverterRule.java   |   13 +-
 .../calcite/adapter/spark/HttpServer.java       |   24 +-
 .../adapter/spark/JdbcToSparkConverter.java     |   54 +-
 .../adapter/spark/JdbcToSparkConverterRule.java |   18 +-
 .../calcite/adapter/spark/SparkHandlerImpl.java |   39 +-
 .../calcite/adapter/spark/SparkMethod.java      |   11 +-
 .../apache/calcite/adapter/spark/SparkRel.java  |   18 +-
 .../calcite/adapter/spark/SparkRules.java       |  168 +-
 .../calcite/adapter/spark/SparkRuntime.java     |   18 +-
 .../spark/SparkToEnumerableConverter.java       |   46 +-
 .../calcite/adapter/spark/package-info.java     |    2 +-
 .../apache/calcite/test/SparkAdapterTest.java   |   35 +-
 splunk/pom.xml                                  |    2 +-
 .../calcite/adapter/splunk/SplunkDriver.java    |   30 +-
 .../adapter/splunk/SplunkDriverVersion.java     |    6 +-
 .../adapter/splunk/SplunkPushDownRule.java      |  135 +-
 .../calcite/adapter/splunk/SplunkQuery.java     |   16 +-
 .../calcite/adapter/splunk/SplunkSchema.java    |   10 +-
 .../calcite/adapter/splunk/SplunkTable.java     |   27 +-
 .../calcite/adapter/splunk/SplunkTableScan.java |   65 +-
 .../calcite/adapter/splunk/package-info.java    |    2 +-
 .../splunk/search/SearchResultListener.java     |    2 +-
 .../adapter/splunk/search/SplunkConnection.java |    4 +-
 .../splunk/search/SplunkConnectionImpl.java     |   33 +-
 .../adapter/splunk/search/package-info.java     |    2 +-
 .../calcite/adapter/splunk/util/HttpUtils.java  |   20 +-
 .../adapter/splunk/util/SocketFactoryImpl.java  |   31 +-
 .../adapter/splunk/util/StringUtils.java        |    2 +-
 .../splunk/util/TrustAllSslSocketFactory.java   |   31 +-
 .../adapter/splunk/util/package-info.java       |    2 +-
 .../apache/calcite/test/SplunkAdapterTest.java  |   90 +-
 src/main/config/checkstyle/checker.xml          |   21 +-
 src/main/config/checkstyle/suppressions.xml     |  100 +-
 .../java/org/apache/calcite/StatementTest.java  |   41 +-
 1236 files changed, 30434 insertions(+), 25248 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/avatica/pom.xml
----------------------------------------------------------------------
diff --git a/avatica/pom.xml b/avatica/pom.xml
index 7220758..b06291d 100644
--- a/avatica/pom.xml
+++ b/avatica/pom.xml
@@ -39,9 +39,7 @@ limitations under the License.
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-surefire-plugin</artifactId>
         <configuration>
-          <excludes>
-            <exclude>net/hydromatic/**/AvaticaTest.java</exclude>
-          </excludes>
+          <excludes/>
         </configuration>
       </plugin>
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/avatica/src/main/java/org/apache/calcite/avatica/ArrayImpl.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/ArrayImpl.java b/avatica/src/main/java/org/apache/calcite/avatica/ArrayImpl.java
index e21ab48..f31aa42 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/ArrayImpl.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/ArrayImpl.java
@@ -14,10 +14,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.avatica;
-
-import java.sql.*;
-import java.util.*;
+package org.apache.calcite.avatica;
+
+import java.sql.Array;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Types;
+import java.util.List;
+import java.util.Map;
 
 /** Implementation of JDBC {@link Array}. */
 public class ArrayImpl implements Array {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/avatica/src/main/java/org/apache/calcite/avatica/AvaticaConnection.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/AvaticaConnection.java b/avatica/src/main/java/org/apache/calcite/avatica/AvaticaConnection.java
index 411e250..ef5aa9a 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/AvaticaConnection.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/AvaticaConnection.java
@@ -14,10 +14,29 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.avatica;
-
-import java.sql.*;
-import java.util.*;
+package org.apache.calcite.avatica;
+
+import java.sql.Array;
+import java.sql.Blob;
+import java.sql.CallableStatement;
+import java.sql.Clob;
+import java.sql.Connection;
+import java.sql.DatabaseMetaData;
+import java.sql.NClob;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLClientInfoException;
+import java.sql.SQLException;
+import java.sql.SQLWarning;
+import java.sql.SQLXML;
+import java.sql.Savepoint;
+import java.sql.Statement;
+import java.sql.Struct;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.TimeZone;
 import java.util.concurrent.Executor;
 
 /**
@@ -422,14 +441,14 @@ public abstract class AvaticaConnection implements Connection {
     }
 
     /** A means for anyone who has a trojan to call the protected method
-     * {@link net.hydromatic.avatica.AvaticaResultSet#execute()}.
+     * {@link org.apache.calcite.avatica.AvaticaResultSet#execute()}.
      * @throws SQLException if execute fails for some reason. */
     public ResultSet execute(AvaticaResultSet resultSet) throws SQLException {
       return resultSet.execute();
     }
 
     /** A means for anyone who has a trojan to call the protected method
-     * {@link net.hydromatic.avatica.AvaticaStatement#getParameterValues()}. */
+     * {@link org.apache.calcite.avatica.AvaticaStatement#getParameterValues()}. */
     public List<Object> getParameterValues(AvaticaStatement statement) {
       return statement.getParameterValues();
     }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/avatica/src/main/java/org/apache/calcite/avatica/AvaticaDatabaseMetaData.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/AvaticaDatabaseMetaData.java b/avatica/src/main/java/org/apache/calcite/avatica/AvaticaDatabaseMetaData.java
index 0e22eff..86e90d7 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/AvaticaDatabaseMetaData.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/AvaticaDatabaseMetaData.java
@@ -14,12 +14,23 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.avatica;
-
-import java.sql.*;
-import java.util.*;
-
-import static net.hydromatic.avatica.InternalProperty.*;
+package org.apache.calcite.avatica;
+
+import java.sql.Connection;
+import java.sql.DatabaseMetaData;
+import java.sql.ResultSet;
+import java.sql.RowIdLifetime;
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.calcite.avatica.InternalProperty.CASE_SENSITIVE;
+import static org.apache.calcite.avatica.InternalProperty.NULL_SORTING;
+import static org.apache.calcite.avatica.InternalProperty.NullSorting;
+import static org.apache.calcite.avatica.InternalProperty.QUOTED_CASING;
+import static org.apache.calcite.avatica.InternalProperty.QUOTING;
+import static org.apache.calcite.avatica.InternalProperty.UNQUOTED_CASING;
 
 /**
  * Implementation of {@link java.sql.DatabaseMetaData}

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/avatica/src/main/java/org/apache/calcite/avatica/AvaticaFactory.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/AvaticaFactory.java b/avatica/src/main/java/org/apache/calcite/avatica/AvaticaFactory.java
index 37610c3..826fb59 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/AvaticaFactory.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/AvaticaFactory.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.avatica;
+package org.apache.calcite.avatica;
 
 import java.sql.ResultSetMetaData;
 import java.sql.SQLException;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/avatica/src/main/java/org/apache/calcite/avatica/AvaticaJdbc40Factory.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/AvaticaJdbc40Factory.java b/avatica/src/main/java/org/apache/calcite/avatica/AvaticaJdbc40Factory.java
index 41006cc..cc2dfaf 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/AvaticaJdbc40Factory.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/AvaticaJdbc40Factory.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.avatica;
+package org.apache.calcite.avatica;
 
 /**
  * Implementation of {@link AvaticaFactory}

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/avatica/src/main/java/org/apache/calcite/avatica/AvaticaJdbc41Factory.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/AvaticaJdbc41Factory.java b/avatica/src/main/java/org/apache/calcite/avatica/AvaticaJdbc41Factory.java
index 219be0c..9e2d269 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/AvaticaJdbc41Factory.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/AvaticaJdbc41Factory.java
@@ -14,11 +14,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.avatica;
+package org.apache.calcite.avatica;
 
 import java.io.InputStream;
 import java.io.Reader;
-import java.sql.*;
+import java.sql.NClob;
+import java.sql.ResultSetMetaData;
+import java.sql.RowId;
+import java.sql.SQLException;
+import java.sql.SQLXML;
 import java.util.List;
 import java.util.Properties;
 import java.util.TimeZone;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/avatica/src/main/java/org/apache/calcite/avatica/AvaticaParameter.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/AvaticaParameter.java b/avatica/src/main/java/org/apache/calcite/avatica/AvaticaParameter.java
index 2b51060..0cf77ae 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/AvaticaParameter.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/AvaticaParameter.java
@@ -14,13 +14,22 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.avatica;
+package org.apache.calcite.avatica;
 
 import java.io.InputStream;
 import java.io.Reader;
 import java.math.BigDecimal;
 import java.net.URL;
-import java.sql.*;
+import java.sql.Array;
+import java.sql.Blob;
+import java.sql.Clob;
+import java.sql.Date;
+import java.sql.NClob;
+import java.sql.Ref;
+import java.sql.RowId;
+import java.sql.SQLXML;
+import java.sql.Time;
+import java.sql.Timestamp;
 import java.util.Calendar;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/avatica/src/main/java/org/apache/calcite/avatica/AvaticaPrepareResult.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/AvaticaPrepareResult.java b/avatica/src/main/java/org/apache/calcite/avatica/AvaticaPrepareResult.java
index 9ce74b2..0836ccf 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/AvaticaPrepareResult.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/AvaticaPrepareResult.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.avatica;
+package org.apache.calcite.avatica;
 
 import java.util.List;
 import java.util.Map;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/avatica/src/main/java/org/apache/calcite/avatica/AvaticaPreparedStatement.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/AvaticaPreparedStatement.java b/avatica/src/main/java/org/apache/calcite/avatica/AvaticaPreparedStatement.java
index 947177a..7b8b9fa 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/AvaticaPreparedStatement.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/AvaticaPreparedStatement.java
@@ -14,13 +14,24 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.avatica;
+package org.apache.calcite.avatica;
 
 import java.io.InputStream;
 import java.io.Reader;
 import java.math.BigDecimal;
 import java.net.URL;
-import java.sql.*;
+import java.sql.Array;
+import java.sql.Blob;
+import java.sql.Clob;
+import java.sql.Date;
+import java.sql.ParameterMetaData;
+import java.sql.PreparedStatement;
+import java.sql.Ref;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+import java.sql.Time;
+import java.sql.Timestamp;
 import java.util.ArrayList;
 import java.util.Calendar;
 import java.util.List;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/avatica/src/main/java/org/apache/calcite/avatica/AvaticaResultSet.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/AvaticaResultSet.java b/avatica/src/main/java/org/apache/calcite/avatica/AvaticaResultSet.java
index 9746ec1..4a2e06e 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/AvaticaResultSet.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/AvaticaResultSet.java
@@ -14,15 +14,30 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.avatica;
+package org.apache.calcite.avatica;
 
 import java.io.InputStream;
 import java.io.Reader;
 import java.math.BigDecimal;
 import java.net.URL;
-import java.sql.*;
+import java.sql.Array;
+import java.sql.Blob;
+import java.sql.Clob;
 import java.sql.Date;
-import java.util.*;
+import java.sql.NClob;
+import java.sql.Ref;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.RowId;
+import java.sql.SQLException;
+import java.sql.SQLWarning;
+import java.sql.SQLXML;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.util.Calendar;
+import java.util.List;
+import java.util.Map;
+import java.util.TimeZone;
 
 /**
  * Implementation of {@link java.sql.ResultSet}
@@ -153,7 +168,7 @@ public class AvaticaResultSet implements ResultSet, ArrayImpl.Factory {
    * constructor occurs while the statement is locked, to make sure that
    * execute/cancel don't happen at the same time.</p>
    *
-   * @see net.hydromatic.avatica.AvaticaConnection.Trojan#execute(AvaticaResultSet)
+   * @see org.apache.calcite.avatica.AvaticaConnection.Trojan#execute(AvaticaResultSet)
    *
    * @throws SQLException if execute fails for some reason.
    */

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/avatica/src/main/java/org/apache/calcite/avatica/AvaticaResultSetMetaData.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/AvaticaResultSetMetaData.java b/avatica/src/main/java/org/apache/calcite/avatica/AvaticaResultSetMetaData.java
index f166102..1043aa8 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/AvaticaResultSetMetaData.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/AvaticaResultSetMetaData.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.avatica;
+package org.apache.calcite.avatica;
 
 import java.sql.ResultSetMetaData;
 import java.sql.SQLException;


[50/58] [abbrv] [partial] incubator-calcite git commit: [CALCITE-306] Standardize code style for "import package.*; "

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/avatica/src/main/java/org/apache/calcite/avatica/AvaticaStatement.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/AvaticaStatement.java b/avatica/src/main/java/org/apache/calcite/avatica/AvaticaStatement.java
index 3a5994d..d1396b1 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/AvaticaStatement.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/AvaticaStatement.java
@@ -14,10 +14,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.avatica;
+package org.apache.calcite.avatica;
 
-import java.sql.*;
-import java.util.*;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.SQLWarning;
+import java.sql.Statement;
+import java.util.Collections;
+import java.util.List;
 
 /**
  * Implementation of {@link java.sql.Statement}
@@ -368,7 +372,7 @@ public abstract class AvaticaStatement
    * <p>The default implementation returns the empty list, because non-prepared
    * statements have no parameters.</p>
    *
-   * @see net.hydromatic.avatica.AvaticaConnection.Trojan#getParameterValues(AvaticaStatement)
+   * @see org.apache.calcite.avatica.AvaticaConnection.Trojan#getParameterValues(AvaticaStatement)
    */
   protected List<Object> getParameterValues() {
     return Collections.emptyList();

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/avatica/src/main/java/org/apache/calcite/avatica/BuiltInConnectionProperty.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/BuiltInConnectionProperty.java b/avatica/src/main/java/org/apache/calcite/avatica/BuiltInConnectionProperty.java
index b70c24c..16501a2 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/BuiltInConnectionProperty.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/BuiltInConnectionProperty.java
@@ -14,13 +14,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.avatica;
+package org.apache.calcite.avatica;
 
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Properties;
 
-import static net.hydromatic.avatica.ConnectionConfigImpl.*;
+import static org.apache.calcite.avatica.ConnectionConfigImpl.PropEnv;
+import static org.apache.calcite.avatica.ConnectionConfigImpl.parse;
 
 /**
  * Enumeration of Avatica's built-in connection properties.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/avatica/src/main/java/org/apache/calcite/avatica/ByteString.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/ByteString.java b/avatica/src/main/java/org/apache/calcite/avatica/ByteString.java
index 7bfdcba..03f04b5 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/ByteString.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/ByteString.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.avatica;
+package org.apache.calcite.avatica;
 
 import java.io.Serializable;
 import java.util.Arrays;
@@ -52,13 +52,11 @@ public class ByteString implements Comparable<ByteString>, Serializable {
     this.bytes = bytes;
   }
 
-  @Override
-  public int hashCode() {
+  @Override public int hashCode() {
     return Arrays.hashCode(bytes);
   }
 
-  @Override
-  public boolean equals(Object obj) {
+  @Override public boolean equals(Object obj) {
     return this == obj
         || obj instanceof ByteString
         && Arrays.equals(bytes, ((ByteString) obj).bytes);
@@ -83,8 +81,7 @@ public class ByteString implements Comparable<ByteString>, Serializable {
    *
    * @return Hexadecimal string
    */
-  @Override
-  public String toString() {
+  @Override public String toString() {
     return toString(16);
   }
 
@@ -141,8 +138,7 @@ public class ByteString implements Comparable<ByteString>, Serializable {
       "CloneDoesntCallSuperClone",
       "CloneDoesntDeclareCloneNotSupportedException"
   })
-  @Override
-  public Object clone() {
+  @Override public Object clone() {
     return this;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/avatica/src/main/java/org/apache/calcite/avatica/Casing.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/Casing.java b/avatica/src/main/java/org/apache/calcite/avatica/Casing.java
index db9e42d..763c514 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/Casing.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/Casing.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.avatica;
+package org.apache.calcite.avatica;
 
 /** Policy for converting case of identifiers before storing them.
  *

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/avatica/src/main/java/org/apache/calcite/avatica/ColumnMetaData.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/ColumnMetaData.java b/avatica/src/main/java/org/apache/calcite/avatica/ColumnMetaData.java
index 598a604..a1997ba 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/ColumnMetaData.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/ColumnMetaData.java
@@ -14,10 +14,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.avatica;
+package org.apache.calcite.avatica;
 
 import java.lang.reflect.Type;
-import java.sql.*;
+import java.sql.DatabaseMetaData;
+import java.sql.ResultSetMetaData;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.sql.Types;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
@@ -25,7 +29,7 @@ import java.util.Map;
 
 /**
  * Metadata for a column.
- * (Compare with {@link java.sql.ResultSetMetaData}.)
+ * (Compare with {@link ResultSetMetaData}.)
  */
 public class ColumnMetaData {
   public final int ordinal; // 0-based

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/avatica/src/main/java/org/apache/calcite/avatica/ConnectStringParser.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/ConnectStringParser.java b/avatica/src/main/java/org/apache/calcite/avatica/ConnectStringParser.java
index c2d68ac..d19815a 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/ConnectStringParser.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/ConnectStringParser.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.avatica;
+package org.apache.calcite.avatica;
 
 import java.sql.SQLException;
 import java.util.Map;
@@ -33,11 +33,15 @@ import java.util.Properties;
  * <li>use of regular {@link Properties} for compatibility with the JDBC API
  * (replaces Mondrian's use of its own order-preserving and case-insensitive
  * PropertyList, found in Util.java at link above)</li>
+ *
  * <li>ability to pass to {@link #parse} a pre-existing Properties object into
  * which properties are to be parsed, possibly overriding prior values</li>
- * <li>use of {@link SQLException}s rather than unchecked {@link
- * RuntimeException}s</li>
+ *
+ * <li>use of {@link SQLException}s rather than unchecked
+ * {@link RuntimeException}s</li>
+ *
  * <li>static members for parsing and creating connect strings</li>
+ *
  * </ul>
  *
  * <p>ConnectStringParser has a private constructor. Callers use the static
@@ -224,7 +228,7 @@ public class ConnectStringParser {
       } else {
         throw new SQLException(
             "quoted value ended too soon, at position " + i
-            + " in '" + s + "'");
+                + " in '" + s + "'");
       }
     } else {
       String value;
@@ -275,8 +279,8 @@ public class ConnectStringParser {
     }
     throw new SQLException(
         "Connect string '" + s
-        + "' contains unterminated quoted value '"
-        + valueBuf.toString() + "'");
+            + "' contains unterminated quoted value '"
+            + valueBuf.toString() + "'");
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/avatica/src/main/java/org/apache/calcite/avatica/ConnectionConfig.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/ConnectionConfig.java b/avatica/src/main/java/org/apache/calcite/avatica/ConnectionConfig.java
index d63e3f3..a799930 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/ConnectionConfig.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/ConnectionConfig.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.avatica;
+package org.apache.calcite.avatica;
 
 /**
  * Connection configuration.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/avatica/src/main/java/org/apache/calcite/avatica/ConnectionConfigImpl.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/ConnectionConfigImpl.java b/avatica/src/main/java/org/apache/calcite/avatica/ConnectionConfigImpl.java
index f406da2..c2c7b45 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/ConnectionConfigImpl.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/ConnectionConfigImpl.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.avatica;
+package org.apache.calcite.avatica;
 
 import java.lang.reflect.Field;
 import java.util.LinkedHashMap;
@@ -39,7 +39,7 @@ public class ConnectionConfigImpl implements ConnectionConfig {
 
   /** Converts a {@link Properties} object containing (name, value)
    * pairs into a map whose keys are
-   * {@link net.hydromatic.avatica.InternalProperty} objects.
+   * {@link org.apache.calcite.avatica.InternalProperty} objects.
    *
    * <p>Matching is case-insensitive. Throws if a property is not known.
    * If a property occurs more than once, takes the last occurrence.</p>

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/avatica/src/main/java/org/apache/calcite/avatica/ConnectionProperty.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/ConnectionProperty.java b/avatica/src/main/java/org/apache/calcite/avatica/ConnectionProperty.java
index 5fcdc50..bfdc4ad 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/ConnectionProperty.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/ConnectionProperty.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.avatica;
+package org.apache.calcite.avatica;
 
 import java.util.Properties;
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/avatica/src/main/java/org/apache/calcite/avatica/Cursor.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/Cursor.java b/avatica/src/main/java/org/apache/calcite/avatica/Cursor.java
index 9207578..b8b42c3 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/Cursor.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/Cursor.java
@@ -14,19 +14,30 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.avatica;
+package org.apache.calcite.avatica;
 
-import java.io.*;
+import java.io.Closeable;
+import java.io.InputStream;
+import java.io.Reader;
 import java.math.BigDecimal;
 import java.net.URL;
-import java.sql.*;
+import java.sql.Array;
+import java.sql.Blob;
+import java.sql.Clob;
+import java.sql.Date;
+import java.sql.NClob;
+import java.sql.Ref;
+import java.sql.SQLException;
+import java.sql.SQLXML;
+import java.sql.Time;
+import java.sql.Timestamp;
 import java.util.Calendar;
 import java.util.List;
 import java.util.Map;
 
 /**
  * Interface to an iteration that is similar to, and can easily support,
- * a JDBC {@link ResultSet}, but is simpler to implement.
+ * a JDBC {@link java.sql.ResultSet}, but is simpler to implement.
  */
 public interface Cursor extends Closeable {
   /**

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/avatica/src/main/java/org/apache/calcite/avatica/DriverVersion.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/DriverVersion.java b/avatica/src/main/java/org/apache/calcite/avatica/DriverVersion.java
index 1f124ee..15c966a 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/DriverVersion.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/DriverVersion.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.avatica;
+package org.apache.calcite.avatica;
 
 import java.io.IOException;
 import java.io.InputStream;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/avatica/src/main/java/org/apache/calcite/avatica/Handler.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/Handler.java b/avatica/src/main/java/org/apache/calcite/avatica/Handler.java
index 0363f4b..831e66d 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/Handler.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/Handler.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.avatica;
+package org.apache.calcite.avatica;
 
 import java.sql.SQLException;
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/avatica/src/main/java/org/apache/calcite/avatica/HandlerImpl.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/HandlerImpl.java b/avatica/src/main/java/org/apache/calcite/avatica/HandlerImpl.java
index 411afd8..c2e6c1a 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/HandlerImpl.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/HandlerImpl.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.avatica;
+package org.apache.calcite.avatica;
 
 import java.sql.SQLException;
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/avatica/src/main/java/org/apache/calcite/avatica/Helper.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/Helper.java b/avatica/src/main/java/org/apache/calcite/avatica/Helper.java
index 8b55c78..3c78b41 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/Helper.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/Helper.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.avatica;
+package org.apache.calcite.avatica;
 
 import java.sql.SQLException;
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/avatica/src/main/java/org/apache/calcite/avatica/InternalProperty.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/InternalProperty.java b/avatica/src/main/java/org/apache/calcite/avatica/InternalProperty.java
index bee4b30..ab5f455 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/InternalProperty.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/InternalProperty.java
@@ -14,15 +14,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.avatica;
+package org.apache.calcite.avatica;
 
-import net.hydromatic.avatica.ConnectionProperty.Type;
+import org.apache.calcite.avatica.ConnectionProperty.Type;
 
-import java.util.*;
+import java.util.Map;
 
 /**
  * Definitions of properties that drive the behavior of
- * {@link net.hydromatic.avatica.AvaticaDatabaseMetaData}.
+ * {@link org.apache.calcite.avatica.AvaticaDatabaseMetaData}.
  */
 public enum InternalProperty {
   /** Whether identifiers are matched case-sensitively. */

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/avatica/src/main/java/org/apache/calcite/avatica/Meta.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/Meta.java b/avatica/src/main/java/org/apache/calcite/avatica/Meta.java
index 4197b76..c552d4e 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/Meta.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/Meta.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.avatica;
+package org.apache.calcite.avatica;
 
 import java.sql.ResultSet;
 import java.util.List;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/avatica/src/main/java/org/apache/calcite/avatica/Quoting.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/Quoting.java b/avatica/src/main/java/org/apache/calcite/avatica/Quoting.java
index 8f49f6c..b73f19c 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/Quoting.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/Quoting.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.avatica;
+package org.apache.calcite.avatica;
 
 /** Syntax for quoting identifiers in SQL statements. */
 public enum Quoting {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/avatica/src/main/java/org/apache/calcite/avatica/UnregisteredDriver.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/UnregisteredDriver.java b/avatica/src/main/java/org/apache/calcite/avatica/UnregisteredDriver.java
index 70e119f..1fcca97 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/UnregisteredDriver.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/UnregisteredDriver.java
@@ -14,10 +14,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.avatica;
-
-import java.sql.*;
-import java.util.*;
+package org.apache.calcite.avatica;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.DriverPropertyInfo;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
@@ -34,8 +42,8 @@ import java.util.logging.Logger;
  *
  * <p>The provider must implement:</p>
  * <ul>
- *   <li>{@link net.hydromatic.avatica.Meta#prepare(AvaticaStatement, String)}</li>
- *   <li>{@link net.hydromatic.avatica.Meta#createCursor(AvaticaResultSet)}</li>
+ *   <li>{@link org.apache.calcite.avatica.Meta#prepare(AvaticaStatement, String)}
+ *   <li>{@link org.apache.calcite.avatica.Meta#createCursor(AvaticaResultSet)}
  * </ul>
  */
 public abstract class UnregisteredDriver implements java.sql.Driver {
@@ -78,12 +86,12 @@ public abstract class UnregisteredDriver implements java.sql.Driver {
   protected String getFactoryClassName(JdbcVersion jdbcVersion) {
     switch (jdbcVersion) {
     case JDBC_30:
-      return "net.hydromatic.avatica.AvaticaFactoryJdbc3Impl";
+      return "org.apache.calcite.avatica.AvaticaFactoryJdbc3Impl";
     case JDBC_40:
-      return "net.hydromatic.avatica.AvaticaJdbc40Factory";
+      return "org.apache.calcite.avatica.AvaticaJdbc40Factory";
     case JDBC_41:
     default:
-      return "net.hydromatic.avatica.AvaticaJdbc41Factory";
+      return "org.apache.calcite.avatica.AvaticaJdbc41Factory";
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/avatica/src/main/java/org/apache/calcite/avatica/package-info.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/package-info.java b/avatica/src/main/java/org/apache/calcite/avatica/package-info.java
index 3b8f3eb..7aad356 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/package-info.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/package-info.java
@@ -18,6 +18,6 @@
 /**
  * Avatica JDBC framework.
  */
-package net.hydromatic.avatica;
+package org.apache.calcite.avatica;
 
 // End package-info.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/avatica/src/test/java/org/apache/calcite/avatica/test/AvaticaSuite.java
----------------------------------------------------------------------
diff --git a/avatica/src/test/java/org/apache/calcite/avatica/test/AvaticaSuite.java b/avatica/src/test/java/org/apache/calcite/avatica/test/AvaticaSuite.java
index c83f40e..fcb3f16 100644
--- a/avatica/src/test/java/org/apache/calcite/avatica/test/AvaticaSuite.java
+++ b/avatica/src/test/java/org/apache/calcite/avatica/test/AvaticaSuite.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.avatica.test;
+package org.apache.calcite.avatica.test;
 
 import org.junit.runner.RunWith;
 import org.junit.runners.Suite;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/avatica/src/test/java/org/apache/calcite/avatica/test/ConnectStringParserTest.java
----------------------------------------------------------------------
diff --git a/avatica/src/test/java/org/apache/calcite/avatica/test/ConnectStringParserTest.java b/avatica/src/test/java/org/apache/calcite/avatica/test/ConnectStringParserTest.java
index bdf7245..cdc8d8a 100644
--- a/avatica/src/test/java/org/apache/calcite/avatica/test/ConnectStringParserTest.java
+++ b/avatica/src/test/java/org/apache/calcite/avatica/test/ConnectStringParserTest.java
@@ -14,16 +14,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.avatica.test;
+package org.apache.calcite.avatica.test;
 
-import net.hydromatic.avatica.ConnectStringParser;
+import org.apache.calcite.avatica.ConnectStringParser;
 
 import org.junit.Test;
 
 import java.sql.SQLException;
 import java.util.Properties;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.fail;
 
 /**
  * Unit test for JDBC connect string parser, {@link ConnectStringParser}. The
@@ -69,8 +71,7 @@ public class ConnectStringParserTest {
    */
   @Test public void testComplexStrings() throws Throwable {
     Properties props =
-        ConnectStringParser.parse(
-            "normalProp=value;"
+        ConnectStringParser.parse("normalProp=value;"
             + "emptyValue=;"
             + " spaceBeforeProp=abc;"
             + " spaceBeforeAndAfterProp =def;"
@@ -226,14 +227,11 @@ public class ConnectStringParserTest {
       Throwable e,
       String expectedPattern) {
     if (e == null) {
-      fail(
-          "Expected an error which matches pattern '" + expectedPattern
-          + "'");
+      fail("Expected an error which matches pattern '" + expectedPattern + "'");
     }
     String msg = e.toString();
     if (!msg.matches(expectedPattern)) {
-      fail(
-          "Got a different error '" + msg + "' than expected '"
+      fail("Got a different error '" + msg + "' than expected '"
           + expectedPattern + "'");
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/avatica/src/test/java/org/apache/calcite/avatica/test/package-info.java
----------------------------------------------------------------------
diff --git a/avatica/src/test/java/org/apache/calcite/avatica/test/package-info.java b/avatica/src/test/java/org/apache/calcite/avatica/test/package-info.java
index 184aef6..7680316 100644
--- a/avatica/src/test/java/org/apache/calcite/avatica/test/package-info.java
+++ b/avatica/src/test/java/org/apache/calcite/avatica/test/package-info.java
@@ -18,6 +18,6 @@
 /**
  * Avatica tests.
  */
-package net.hydromatic.avatica.test;
+package org.apache.calcite.avatica.test;
 
 // End package-info.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/pom.xml
----------------------------------------------------------------------
diff --git a/core/pom.xml b/core/pom.xml
index 116fab2..1c5527c 100644
--- a/core/pom.xml
+++ b/core/pom.xml
@@ -58,7 +58,7 @@ limitations under the License.
         <artifactId>maven-surefire-plugin</artifactId>
         <configuration>
           <includes>
-            <include>net/hydromatic/optiq/test/OptiqSuite.java</include>
+            <include>org/apache/calcite/test/CalciteSuite.java</include>
           </includes>
           <threadCount>1</threadCount>
           <perCoreThreadCount>true</perCoreThreadCount>
@@ -78,7 +78,7 @@ limitations under the License.
             <configuration>
               <sourceDirectory>${project.build.directory}/generated-sources/fmpp</sourceDirectory>
               <includes>
-                <include>**/CombinedParser.jj</include>
+                <include>**/Parser.jj</include>
               </includes>
               <lookAhead>2</lookAhead>
               <isStatic>false</isStatic>
@@ -95,7 +95,7 @@ limitations under the License.
               <goal>generate-sources</goal>
             </goals>
             <configuration>
-              <packageName>org.eigenbase.resource</packageName>
+              <packageName>org.apache.calcite.runtime</packageName>
             </configuration>
           </execution>
         </executions>

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/codegen/config.fmpp
----------------------------------------------------------------------
diff --git a/core/src/main/codegen/config.fmpp b/core/src/main/codegen/config.fmpp
index b45cd3a..d480488 100644
--- a/core/src/main/codegen/config.fmpp
+++ b/core/src/main/codegen/config.fmpp
@@ -17,7 +17,7 @@
 # allow clients to extend Calcite's SQL parser to support application specific
 # SQL statements, literals or data types.
 #
-# Calcite's parser grammar file (CombinedParser.jj) is written in javacc
+# Calcite's parser grammar file (Parser.jj) is written in javacc
 # (http://javacc.java.net/) with Freemarker (http://freemarker.org/) variables
 # to allow clients to:
 #   1. have custom parser implementation class and package name.
@@ -29,17 +29,17 @@
 #   3. add new keywords to support custom SQL constructs added as part of (2).
 #   4. add import statements needed by inserted custom parser implementations.
 #
-# Parser template file (CombinerParser.jj) along with this file are packaged as
+# Parser template file (Parser.jj) along with this file are packaged as
 # part of the calcite-core-<version>.jar under "codegen" directory.
 
 data: {
   parser: {
     # Generated parser implementation package and class name.
-    package: "org.eigenbase.sql.parser.impl",
+    package: "org.apache.calcite.sql.parser.impl",
     class: "SqlParserImpl",
 
     # List of additional classes and packages to import.
-    # Example. "org.eigenbase.sql.*", "java.util.List".
+    # Example. "org.apache.calcite.sql.*", "java.util.List".
     imports: [
     ]
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 54b6df2..de98351 100644
--- a/core/src/main/codegen/templates/Parser.jj
+++ b/core/src/main/codegen/templates/Parser.jj
@@ -16,7 +16,7 @@
  */
 <@pp.dropOutputFile />
 
-<@pp.changeOutputFile name="javacc/CombinedParser.jj" />
+<@pp.changeOutputFile name="javacc/Parser.jj" />
 
 options {
     STATIC = false;
@@ -33,41 +33,85 @@ package ${parser.package};
 import ${importStr};
 </#list>
 
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.sql.fun.*;
-import org.eigenbase.trace.EigenbaseTrace;
-import org.eigenbase.util.BitString;
-import org.eigenbase.util14.DateTimeUtil;
-import org.eigenbase.sql.fun.SqlStdOperatorTable;
-import org.eigenbase.sql.type.SqlTypeName;
-import org.eigenbase.resource.*;
 
-import net.hydromatic.avatica.Casing;
+import org.apache.calcite.avatica.Casing;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.runtime.CalciteContextException;
+import org.apache.calcite.sql.JoinConditionType;
+import org.apache.calcite.sql.JoinType;
+import org.apache.calcite.sql.SqlBinaryOperator;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlCharStringLiteral;
+import org.apache.calcite.sql.SqlCollation;
+import org.apache.calcite.sql.SqlDataTypeSpec;
+import org.apache.calcite.sql.SqlDateLiteral;
+import org.apache.calcite.sql.SqlDelete;
+import org.apache.calcite.sql.SqlDynamicParam;
+import org.apache.calcite.sql.SqlExplain;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlInsert;
+import org.apache.calcite.sql.SqlIntervalLiteral;
+import org.apache.calcite.sql.SqlIntervalQualifier;
+import org.apache.calcite.sql.SqlJdbcFunctionCall;
+import org.apache.calcite.sql.SqlJoin;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlMerge;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlNodeList;
+import org.apache.calcite.sql.SqlNumericLiteral;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlOrderBy;
+import org.apache.calcite.sql.SqlPostfixOperator;
+import org.apache.calcite.sql.SqlPrefixOperator;
+import org.apache.calcite.sql.SqlSampleSpec;
+import org.apache.calcite.sql.SqlSelect;
+import org.apache.calcite.sql.SqlSelectKeyword;
+import org.apache.calcite.sql.SqlSetOption;
+import org.apache.calcite.sql.SqlTimeLiteral;
+import org.apache.calcite.sql.SqlTimestampLiteral;
+import org.apache.calcite.sql.SqlUpdate;
+import org.apache.calcite.sql.SqlUtil;
+import org.apache.calcite.sql.SqlWindow;
+import org.apache.calcite.sql.SqlWith;
+import org.apache.calcite.sql.SqlWithItem;
+import org.apache.calcite.sql.fun.SqlCase;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.fun.SqlTrimFunction;
+import org.apache.calcite.sql.parser.SqlAbstractParserImpl;
+import org.apache.calcite.sql.parser.SqlParseException;
+import org.apache.calcite.sql.parser.SqlParser;
+import org.apache.calcite.sql.parser.SqlParserImplFactory;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.parser.SqlParserUtil;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.util.trace.CalciteTrace;
+import org.apache.calcite.util.DateTimeUtil;
 
 import com.google.common.collect.ImmutableList;
 
 import java.io.Reader;
-import java.math.*;
+import java.math.BigDecimal;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Calendar;
 import java.util.Collections;
 import java.util.List;
-import java.util.logging.Logger;
 import java.util.logging.Level;
+import java.util.logging.Logger;
 
-import static org.eigenbase.util.Static.RESOURCE;
+import static org.apache.calcite.util.Static.RESOURCE;
 
 /**
- * SQL parser, generated from Parser.jj and CommonParser.jj by JavaCC.
+ * SQL parser, generated from Parser.jj by JavaCC.
  *
  * <p>The public wrapper for this parser is {@link SqlParser}.
  */
 public class ${parser.class} extends SqlAbstractParserImpl
 {
-    private static final Logger LOGGER = EigenbaseTrace.getParserTracer();
+    private static final Logger LOGGER = CalciteTrace.getParserTracer();
 
     // Can't use quoted literal because of a bug in how JavaCC translates
     // backslash-backslash.
@@ -437,15 +481,15 @@ JAVACODE SqlParseException convertException(Throwable ex)
             int column = Integer.parseInt(matcher.group(2));
             pos = new SqlParserPos(line, column, line, column);
         }
-    } else if (ex instanceof org.eigenbase.util.EigenbaseContextException) {
-        // EigenbaseContextException is the standard wrapper for exceptions
+    } else if (ex instanceof CalciteContextException) {
+        // CalciteContextException is the standard wrapper for exceptions
         // produced by the validator, but in the parser, the standard is
         // SqlParseException; so, strip it away. In case you were wondering,
-        // the EigenbaseContextException appears because the parser
+        // the CalciteContextException appears because the parser
         // occasionally calls into validator-style code such as
         // SqlSpecialOperator.reduceExpr.
-        org.eigenbase.util.EigenbaseContextException ece =
-            (org.eigenbase.util.EigenbaseContextException) ex;
+        CalciteContextException ece =
+            (CalciteContextException) ex;
         pos = new SqlParserPos(
             ece.getPosLine(),
             ece.getPosColumn(),
@@ -666,9 +710,7 @@ SqlNode ParenthesizedExpression(ExprContext exprContext) :
 /**
  * Parses a parenthesized query or comma-list of row expressions.
  *
- *<p>
- *
- * REVIEW jvs 8-Feb-2004: There's a small hole in this production.  It can be
+ * <p>REVIEW jvs 8-Feb-2004: There's a small hole in this production.  It can be
  * used to construct something like
  *
  * <code>WHERE x IN (select count(*) from t where c=d,5)</code>,
@@ -732,10 +774,8 @@ SqlNodeList ParenthesizedQueryOrCommaList(
 /**
  * Parses function parameter lists including DISTINCT keyword recognition
  *
- *<p>
- *
- * This is pretty much the same as ParenthesizedQueryOrCommaList but allows the
- * DISTINCT keyword to follow the left paren and not be followed by a comma.
+ * <p>This is pretty much the same as ParenthesizedQueryOrCommaList but allows
+ * the DISTINCT keyword to follow the left paren and not be followed by a comma.
  */
 List FunctionParameterList(
     ExprContext exprContext) :
@@ -3919,9 +3959,7 @@ SqlNode BuiltinFunctionCall() :
  * Parses a call to a named function (could be a builtin with regular
  * syntax, or else a UDF).
  *
- *<p>
- *
- * NOTE: every UDF has two names: an <em>invocation name</em> and a
+ * <p>NOTE: every UDF has two names: an <em>invocation name</em> and a
  * <em>specific name</em>.  Normally, function calls are resolved via overload
  * resolution and invocation names.  The SPECIFIC prefix allows overload
  * resolution to be bypassed.  Note that usage of the SPECIFIC prefix in
@@ -3929,9 +3967,7 @@ SqlNode BuiltinFunctionCall() :
  * view definitions to permanently bind references to a particular function
  * after the overload resolution performed by view creation.
  *
- *<p>
- *
- * TODO jvs 25-Mar-2005:  Once we have SQL-Flagger support, flag SPECIFIC
+ * <p>TODO jvs 25-Mar-2005:  Once we have SQL-Flagger support, flag SPECIFIC
  * as non-standard.
  */
 SqlNode NamedFunctionCall() :
@@ -4916,9 +4952,7 @@ SqlPostfixOperator PostfixRowOperator() :
  * of this parser can use this as a base for implementing the
  * NonReservedKeyWord() production.
  *
- *<p>
- *
- * When adding keywords to this list, be sure that they are not reserved
+ * <p>When adding keywords to this list, be sure that they are not reserved
  * by the SQL:2003 standard (see productions for "non-reserved word"
  * and "reserved word" in reference below).
  *

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/DataContext.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/DataContext.java b/core/src/main/java/org/apache/calcite/DataContext.java
index 1f92631..3d5203f 100644
--- a/core/src/main/java/org/apache/calcite/DataContext.java
+++ b/core/src/main/java/org/apache/calcite/DataContext.java
@@ -14,16 +14,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq;
+package org.apache.calcite;
 
-import net.hydromatic.linq4j.QueryProvider;
-import net.hydromatic.linq4j.expressions.Expressions;
-import net.hydromatic.linq4j.expressions.ParameterExpression;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.linq4j.QueryProvider;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.linq4j.tree.ParameterExpression;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.sql.advise.SqlAdvisor;
 
-import net.hydromatic.optiq.impl.java.JavaTypeFactory;
-
-import org.eigenbase.sql.advise.SqlAdvisor;
-import org.eigenbase.util.Util;
+import com.google.common.base.CaseFormat;
 
 import java.lang.reflect.Modifier;
 import java.util.TimeZone;
@@ -90,7 +90,8 @@ public interface DataContext {
     Variable(String camelName, Class clazz) {
       this.camelName = camelName;
       this.clazz = clazz;
-      assert camelName.equals(Util.toCamelCase(name()));
+      assert camelName.equals(
+          CaseFormat.UPPER_UNDERSCORE.to(CaseFormat.LOWER_CAMEL, name()));
     }
 
     /** Returns the value of this variable in a given data context. */

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/Demo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/Demo.java b/core/src/main/java/org/apache/calcite/Demo.java
index 03ea39e..27f65fe 100644
--- a/core/src/main/java/org/apache/calcite/Demo.java
+++ b/core/src/main/java/org/apache/calcite/Demo.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq;
+package org.apache.calcite;
 
 import java.util.ArrayList;
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/clone/ArrayTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/clone/ArrayTable.java b/core/src/main/java/org/apache/calcite/adapter/clone/ArrayTable.java
index b97bb18..6bfe42f 100644
--- a/core/src/main/java/org/apache/calcite/adapter/clone/ArrayTable.java
+++ b/core/src/main/java/org/apache/calcite/adapter/clone/ArrayTable.java
@@ -14,25 +14,35 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl.clone;
-
-import net.hydromatic.linq4j.*;
-import net.hydromatic.linq4j.expressions.Primitive;
-
-import net.hydromatic.optiq.*;
-import net.hydromatic.optiq.impl.AbstractTableQueryable;
-import net.hydromatic.optiq.impl.java.AbstractQueryableTable;
-import net.hydromatic.optiq.util.BitSets;
-
-import org.eigenbase.reltype.*;
-import org.eigenbase.util.*;
+package org.apache.calcite.adapter.clone;
+
+import org.apache.calcite.adapter.java.AbstractQueryableTable;
+import org.apache.calcite.linq4j.Enumerator;
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.linq4j.QueryProvider;
+import org.apache.calcite.linq4j.Queryable;
+import org.apache.calcite.linq4j.tree.Primitive;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelProtoDataType;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.schema.Statistic;
+import org.apache.calcite.schema.Statistics;
+import org.apache.calcite.schema.impl.AbstractTableQueryable;
+import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.Pair;
 
 import com.google.common.base.Supplier;
 import com.google.common.collect.ImmutableList;
 
 import java.lang.reflect.Array;
 import java.lang.reflect.Type;
-import java.util.*;
+import java.util.AbstractList;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.BitSet;
+import java.util.Collections;
+import java.util.List;
 
 /**
  * Implementation of table that reads rows from column stores, one per column.
@@ -185,8 +195,7 @@ class ArrayTable extends AbstractQueryableTable {
           cardinality);
     }
 
-    @Override
-    public String toString() {
+    @Override public String toString() {
       return "Column(representation=" + representation
           + ", value=" + representation.toString(dataSet) + ")";
     }
@@ -453,8 +462,7 @@ class ArrayTable extends AbstractQueryableTable {
     public StringDictionary() {
     }
 
-    @Override
-    public String toString() {
+    @Override public String toString() {
       return "StringDictionary()";
     }
 
@@ -591,8 +599,7 @@ class ArrayTable extends AbstractQueryableTable {
       this.signed = signed;
     }
 
-    @Override
-    public String toString() {
+    @Override public String toString() {
       return "BitSlicedPrimitiveArray(ordinal=" + ordinal
           + ", bitCount=" + bitCount
           + ", primitive=" + primitive

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/clone/CloneSchema.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/clone/CloneSchema.java b/core/src/main/java/org/apache/calcite/adapter/clone/CloneSchema.java
index 2f37b43..2f996b9 100644
--- a/core/src/main/java/org/apache/calcite/adapter/clone/CloneSchema.java
+++ b/core/src/main/java/org/apache/calcite/adapter/clone/CloneSchema.java
@@ -14,19 +14,23 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl.clone;
+package org.apache.calcite.adapter.clone;
 
-import net.hydromatic.avatica.ColumnMetaData;
-
-import net.hydromatic.linq4j.*;
-
-import net.hydromatic.optiq.*;
-import net.hydromatic.optiq.impl.AbstractSchema;
-import net.hydromatic.optiq.impl.java.*;
-import net.hydromatic.optiq.impl.jdbc.JdbcSchema;
-import net.hydromatic.optiq.jdbc.OptiqConnection;
-
-import org.eigenbase.reltype.RelProtoDataType;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.adapter.jdbc.JdbcSchema;
+import org.apache.calcite.avatica.ColumnMetaData;
+import org.apache.calcite.jdbc.CalciteConnection;
+import org.apache.calcite.linq4j.Enumerable;
+import org.apache.calcite.linq4j.QueryProvider;
+import org.apache.calcite.linq4j.Queryable;
+import org.apache.calcite.rel.type.RelProtoDataType;
+import org.apache.calcite.schema.QueryableTable;
+import org.apache.calcite.schema.Schema;
+import org.apache.calcite.schema.SchemaFactory;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.schema.Schemas;
+import org.apache.calcite.schema.Table;
+import org.apache.calcite.schema.impl.AbstractSchema;
 
 import com.google.common.base.Supplier;
 import com.google.common.base.Suppliers;
@@ -36,7 +40,7 @@ import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 
-import static net.hydromatic.optiq.impl.MaterializedViewTable.MATERIALIZATION_CONNECTION;
+import static org.apache.calcite.schema.impl.MaterializedViewTable.MATERIALIZATION_CONNECTION;
 
 /**
  * Schema that contains in-memory copies of tables from a JDBC schema.
@@ -58,8 +62,7 @@ public class CloneSchema extends AbstractSchema {
     this.sourceSchema = sourceSchema;
   }
 
-  @Override
-  protected Map<String, Table> getTableMap() {
+  @Override protected Map<String, Table> getTableMap() {
     final Map<String, Table> map = new LinkedHashMap<String, Table>();
     for (String name : sourceSchema.getTableNames()) {
       final Table table = sourceSchema.getTable(name);
@@ -77,7 +80,7 @@ public class CloneSchema extends AbstractSchema {
     final Queryable<Object> queryable =
         sourceTable.asQueryable(queryProvider, sourceSchema, name);
     final JavaTypeFactory typeFactory =
-        ((OptiqConnection) queryProvider).getTypeFactory();
+        ((CalciteConnection) queryProvider).getTypeFactory();
     return createCloneTable(typeFactory, Schemas.proto(sourceTable), null,
         queryable);
   }
@@ -114,7 +117,7 @@ public class CloneSchema extends AbstractSchema {
   }
 
   /** Schema factory that creates a
-   * {@link net.hydromatic.optiq.impl.clone.CloneSchema}.
+   * {@link org.apache.calcite.adapter.clone.CloneSchema}.
    * This allows you to create a clone schema inside a model.json file.
    *
    * <pre>{@code
@@ -125,7 +128,7 @@ public class CloneSchema extends AbstractSchema {
    *     {
    *       name: 'FOODMART_CLONE',
    *       type: 'custom',
-   *       factory: 'net.hydromatic.optiq.impl.clone.CloneSchema$Factory',
+   *       factory: 'org.apache.calcite.adapter.clone.CloneSchema$Factory',
    *       operand: {
    *         jdbcDriver: 'com.mysql.jdbc.Driver',
    *         jdbcUrl: 'jdbc:mysql://localhost/foodmart',

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/clone/ColumnLoader.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/clone/ColumnLoader.java b/core/src/main/java/org/apache/calcite/adapter/clone/ColumnLoader.java
index 63a7ba7..1338fce 100644
--- a/core/src/main/java/org/apache/calcite/adapter/clone/ColumnLoader.java
+++ b/core/src/main/java/org/apache/calcite/adapter/clone/ColumnLoader.java
@@ -14,28 +14,31 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl.clone;
-
-import net.hydromatic.avatica.ColumnMetaData;
-
-import net.hydromatic.linq4j.Enumerable;
-import net.hydromatic.linq4j.Ord;
-import net.hydromatic.linq4j.expressions.Primitive;
-import net.hydromatic.linq4j.function.Function1;
-import net.hydromatic.linq4j.function.Functions;
-
-import net.hydromatic.optiq.impl.java.JavaTypeFactory;
-
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.reltype.RelDataTypeField;
-import org.eigenbase.reltype.RelProtoDataType;
-import org.eigenbase.util14.DateTimeUtil;
+package org.apache.calcite.adapter.clone;
+
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.avatica.ColumnMetaData;
+import org.apache.calcite.linq4j.Enumerable;
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.linq4j.function.Function1;
+import org.apache.calcite.linq4j.function.Functions;
+import org.apache.calcite.linq4j.tree.Primitive;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rel.type.RelProtoDataType;
+import org.apache.calcite.util.DateTimeUtil;
 
 import java.lang.reflect.Type;
 import java.sql.Date;
 import java.sql.Time;
 import java.sql.Timestamp;
-import java.util.*;
+import java.util.AbstractList;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
 
 /**
  * Column loader.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/clone/ListTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/clone/ListTable.java b/core/src/main/java/org/apache/calcite/adapter/clone/ListTable.java
index 34b6abd..c07ea0a 100644
--- a/core/src/main/java/org/apache/calcite/adapter/clone/ListTable.java
+++ b/core/src/main/java/org/apache/calcite/adapter/clone/ListTable.java
@@ -14,20 +14,27 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl.clone;
+package org.apache.calcite.adapter.clone;
 
-import net.hydromatic.linq4j.*;
-import net.hydromatic.linq4j.expressions.Expression;
-
-import net.hydromatic.optiq.*;
-import net.hydromatic.optiq.impl.java.AbstractQueryableTable;
-
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.reltype.RelDataTypeFactory;
-import org.eigenbase.reltype.RelProtoDataType;
+import org.apache.calcite.adapter.java.AbstractQueryableTable;
+import org.apache.calcite.linq4j.AbstractQueryable;
+import org.apache.calcite.linq4j.Enumerator;
+import org.apache.calcite.linq4j.Linq4j;
+import org.apache.calcite.linq4j.QueryProvider;
+import org.apache.calcite.linq4j.Queryable;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelProtoDataType;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.schema.Statistic;
+import org.apache.calcite.schema.Statistics;
 
 import java.lang.reflect.Type;
-import java.util.*;
+import java.util.BitSet;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
 
 /**
  * Implementation of table that reads rows from a read-only list and returns

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/clone/package-info.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/clone/package-info.java b/core/src/main/java/org/apache/calcite/adapter/clone/package-info.java
index 1c29715..eef12e2 100644
--- a/core/src/main/java/org/apache/calcite/adapter/clone/package-info.java
+++ b/core/src/main/java/org/apache/calcite/adapter/clone/package-info.java
@@ -18,6 +18,6 @@
 /**
  * Provides utility classes.
  */
-package net.hydromatic.optiq.impl.clone;
+package org.apache.calcite.adapter.clone;
 
 // End package-info.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/AggAddContext.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/AggAddContext.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/AggAddContext.java
index 6d8c4c1..a66e2d7 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/AggAddContext.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/AggAddContext.java
@@ -14,25 +14,26 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.rules.java;
+package org.apache.calcite.adapter.enumerable;
 
-import net.hydromatic.linq4j.expressions.Expression;
-
-import org.eigenbase.rex.RexNode;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.rex.RexNode;
 
 import java.util.List;
 
 /**
- * Information for a call to {@link net.hydromatic.optiq.rules.java.AggImplementor#implementAdd(AggContext, AggAddContext)}.
- * Typically, the aggregation implementation will use {@link #arguments()}
+ * Information for a call to
+ * {@link org.apache.calcite.adapter.enumerable.AggImplementor#implementAdd(AggContext, AggAddContext)}.
+ *
+ * <p>Typically, the aggregation implementation will use {@link #arguments()}
  * or {@link #rexArguments()} to update aggregate value.
  */
 public interface AggAddContext extends AggResultContext {
   /**
-   * Returns {@link org.eigenbase.rex.RexNode} representation of arguments.
+   * Returns {@link org.apache.calcite.rex.RexNode} representation of arguments.
    * This can be useful for manual translation of required arguments with
    * different {@link NullPolicy}.
-   * @return {@link org.eigenbase.rex.RexNode} representation of arguments
+   * @return {@link org.apache.calcite.rex.RexNode} representation of arguments
    */
   List<RexNode> rexArguments();
 
@@ -46,8 +47,9 @@ public interface AggAddContext extends AggResultContext {
   List<Expression> arguments();
 
   /**
-   * Returns {@link net.hydromatic.optiq.rules.java.RexToLixTranslator} suitable to transform the arguments.
-   * @return {@link net.hydromatic.optiq.rules.java.RexToLixTranslator} suitable to transform the arguments.
+   * Returns a
+   * {@link org.apache.calcite.adapter.enumerable.RexToLixTranslator}
+   * suitable to transform the arguments.
    */
   RexToLixTranslator rowTranslator();
 }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/AggContext.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/AggContext.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/AggContext.java
index 5582c50..a72471a 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/AggContext.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/AggContext.java
@@ -14,10 +14,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.rules.java;
+package org.apache.calcite.adapter.enumerable;
 
-import org.eigenbase.rel.Aggregation;
-import org.eigenbase.reltype.RelDataType;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlAggFunction;
 
 import java.lang.reflect.Type;
 import java.util.List;
@@ -32,12 +32,15 @@ public interface AggContext {
    * Returns the aggregation being implemented.
    * @return aggregation being implemented.
    */
-  Aggregation aggregation();
+  SqlAggFunction aggregation();
 
   /**
-   * Returns the return type of the aggregate as {@link org.eigenbase.reltype.RelDataType}.
-   * This can be helpful to test {@link org.eigenbase.reltype.RelDataType#isNullable()}.
-   * @return return type of the aggregate as {@link org.eigenbase.reltype.RelDataType}
+   * Returns the return type of the aggregate as
+   * {@link org.apache.calcite.rel.type.RelDataType}.
+   * This can be helpful to test
+   * {@link org.apache.calcite.rel.type.RelDataType#isNullable()}.
+   *
+   * @return return type of the aggregate
    */
   RelDataType returnRelType();
 
@@ -48,15 +51,18 @@ public interface AggContext {
   Type returnType();
 
   /**
-   * Returns the parameter types of the aggregate as {@link org.eigenbase.reltype.RelDataType}.
-   * This can be helpful to test {@link org.eigenbase.reltype.RelDataType#isNullable()}.
-   * @return parameter types of the aggregate as {@link org.eigenbase.reltype.RelDataType}
+   * Returns the parameter types of the aggregate as
+   * {@link org.apache.calcite.rel.type.RelDataType}.
+   * This can be helpful to test
+   * {@link org.apache.calcite.rel.type.RelDataType#isNullable()}.
    */
   List<? extends RelDataType> parameterRelTypes();
 
   /**
-   * Returns the parameter types of the aggregate as {@link java.lang.reflect.Type}.
-   * @return parameter types of the aggregate as {@link java.lang.reflect.Type}
+   * Returns the parameter types of the aggregate as
+   * {@link java.lang.reflect.Type}.
    */
   List<? extends Type> parameterTypes();
 }
+
+// End AggContext.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/AggImpState.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/AggImpState.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/AggImpState.java
index 5301993..1c8909a 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/AggImpState.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/AggImpState.java
@@ -14,11 +14,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.rules.java;
+package org.apache.calcite.adapter.enumerable;
 
-import net.hydromatic.linq4j.expressions.Expression;
-
-import org.eigenbase.rel.AggregateCall;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.rel.core.AggregateCall;
 
 import java.util.List;
 
@@ -46,3 +45,5 @@ class AggImpState {
     }
   }
 }
+
+// End AggImpState.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/AggImplementor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/AggImplementor.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/AggImplementor.java
index b4e2f17..6f70828 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/AggImplementor.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/AggImplementor.java
@@ -14,9 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.rules.java;
+package org.apache.calcite.adapter.enumerable;
 
-import net.hydromatic.linq4j.expressions.Expression;
+import org.apache.calcite.linq4j.tree.Expression;
 
 import java.lang.reflect.Type;
 import java.util.List;
@@ -25,17 +25,20 @@ import java.util.List;
  * Implements an aggregate function by generating expressions to
  * initialize, add to, and get a result from, an accumulator.
  *
- * @see net.hydromatic.optiq.rules.java.StrictAggImplementor
- * @see net.hydromatic.optiq.rules.java.StrictWinAggImplementor
- * @see net.hydromatic.optiq.rules.java.RexImpTable.CountImplementor
- * @see net.hydromatic.optiq.rules.java.RexImpTable.SumImplementor
+ * @see org.apache.calcite.adapter.enumerable.StrictAggImplementor
+ * @see org.apache.calcite.adapter.enumerable.StrictWinAggImplementor
+ * @see org.apache.calcite.adapter.enumerable.RexImpTable.CountImplementor
+ * @see org.apache.calcite.adapter.enumerable.RexImpTable.SumImplementor
  */
 public interface AggImplementor {
   /**
    * Returns the types of the intermediate variables used by the aggregate
    * implementation.
-   * For instance, for "concatenate to string" this can be {@link java.lang.StringBuilder}.
+   *
+   * <p>For instance, for "concatenate to string" this can be
+   * {@link java.lang.StringBuilder}.
    * Calcite calls this method before all other {@code implement*} methods.
+   *
    * @param info aggregate context
    * @return types of the intermediate variables used by the aggregate
    *   implementation

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/AggResetContext.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/AggResetContext.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/AggResetContext.java
index 2e474cd..b3efb21 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/AggResetContext.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/AggResetContext.java
@@ -14,14 +14,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.rules.java;
+package org.apache.calcite.adapter.enumerable;
 
-import net.hydromatic.linq4j.expressions.Expression;
+import org.apache.calcite.linq4j.tree.Expression;
 
 import java.util.List;
 
 /**
- * Information for a call to {@link AggImplementor#implementReset(AggContext, AggResetContext)}.
+ * Information for a call to
+ * {@link AggImplementor#implementReset(AggContext, AggResetContext)}.
+ *
  * {@link AggResetContext} provides access to the accumulator variables
  * that should be reset.
  */
@@ -29,9 +31,11 @@ public interface AggResetContext extends NestedBlockBuilder {
   /**
    * Returns accumulator variables that should be reset.
    * There MUST be an assignment even if you just assign the default value.
+   *
    * @return accumulator variables that should be reset or empty list when no
    *   accumulator variables are used by the aggregate implementation.
-   * @see AggImplementor#getStateType(net.hydromatic.optiq.rules.java.AggContext)
+   *
+   * @see AggImplementor#getStateType(org.apache.calcite.adapter.enumerable.AggContext)
    */
   List<Expression> accumulator();
 }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/AggResultContext.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/AggResultContext.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/AggResultContext.java
index 0a0de5a..fe7ee06 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/AggResultContext.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/AggResultContext.java
@@ -14,13 +14,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.rules.java;
+package org.apache.calcite.adapter.enumerable;
 
 /**
- * Information for a call to {@link AggImplementor#implementResult(AggContext, AggResultContext)}
- * Typically, the aggregation implementation will convert {@link #accumulator()}
- * to the resulting value of the aggregation.
- * The implementation MUST NOT destroy the contents of {@link #accumulator()}.
+ * Information for a call to
+ * {@link AggImplementor#implementResult(AggContext, AggResultContext)}
+ *
+ * <p>Typically, the aggregation implementation will convert
+ * {@link #accumulator()} to the resulting value of the aggregation.  The
+ * implementation MUST NOT destroy the contents of {@link #accumulator()}.
  */
 public interface AggResultContext extends NestedBlockBuilder, AggResetContext {
 }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/CallImplementor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/CallImplementor.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/CallImplementor.java
index ecc1b5d..8f8ea68 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/CallImplementor.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/CallImplementor.java
@@ -14,18 +14,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.rules.java;
+package org.apache.calcite.adapter.enumerable;
 
-import net.hydromatic.linq4j.expressions.Expression;
-
-import org.eigenbase.rex.RexCall;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.rex.RexCall;
 
 /**
  * Implements a call via given translator.
- * <p>
- * @see net.hydromatic.optiq.ScalarFunction
- * @see net.hydromatic.optiq.TableFunction
- * @see net.hydromatic.optiq.rules.java.RexImpTable
+ *
+ * @see org.apache.calcite.schema.ScalarFunction
+ * @see org.apache.calcite.schema.TableFunction
+ * @see org.apache.calcite.adapter.enumerable.RexImpTable
  */
 public interface CallImplementor {
   /** Implements a call. */
@@ -34,3 +33,5 @@ public interface CallImplementor {
       RexCall call,
       RexImpTable.NullAs nullAs);
 }
+
+// End CallImplementor.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumUtils.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumUtils.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumUtils.java
new file mode 100644
index 0000000..e025c5a
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumUtils.java
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.linq4j.tree.BlockStatement;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.linq4j.tree.MethodDeclaration;
+import org.apache.calcite.linq4j.tree.ParameterExpression;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexNode;
+
+import com.google.common.collect.ImmutableList;
+
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.lang.reflect.Type;
+import java.util.AbstractList;
+import java.util.List;
+
+/**
+ * Utilities for generating programs in the Enumerable (functional)
+ * style.
+ */
+public class EnumUtils {
+  private EnumUtils() {}
+
+  static final boolean BRIDGE_METHODS = true;
+
+  static final List<ParameterExpression> NO_PARAMS =
+      ImmutableList.of();
+
+  static final List<Expression> NO_EXPRS =
+      ImmutableList.of();
+
+  public static final String[] LEFT_RIGHT = {"left", "right"};
+
+  static final boolean B = false;
+
+  /** Declares a method that overrides another method. */
+  public static MethodDeclaration overridingMethodDecl(Method method,
+      Iterable<ParameterExpression> parameters,
+      BlockStatement body) {
+    return Expressions.methodDecl(
+        method.getModifiers() & ~Modifier.ABSTRACT,
+        method.getReturnType(),
+        method.getName(),
+        parameters,
+        body);
+  }
+
+  static Type javaClass(
+      JavaTypeFactory typeFactory, RelDataType type) {
+    final Type clazz = typeFactory.getJavaClass(type);
+    return clazz instanceof Class ? clazz : Object[].class;
+  }
+
+  static Class javaRowClass(
+      JavaTypeFactory typeFactory, RelDataType type) {
+    if (type.isStruct() && type.getFieldCount() == 1) {
+      type = type.getFieldList().get(0).getType();
+    }
+    final Type clazz = typeFactory.getJavaClass(type);
+    return clazz instanceof Class ? (Class) clazz : Object[].class;
+  }
+
+  static List<Type> fieldTypes(
+      final JavaTypeFactory typeFactory,
+      final List<? extends RelDataType> inputTypes) {
+    return new AbstractList<Type>() {
+      public Type get(int index) {
+        return EnumUtils.javaClass(typeFactory, inputTypes.get(index));
+      }
+      public int size() {
+        return inputTypes.size();
+      }
+    };
+  }
+
+  static List<RelDataType> fieldRowTypes(
+      final RelDataType inputRowType,
+      final List<? extends RexNode> extraInputs,
+      final List<Integer> argList) {
+    final List<RelDataTypeField> inputFields = inputRowType.getFieldList();
+    return new AbstractList<RelDataType>() {
+      public RelDataType get(int index) {
+        final int arg = argList.get(index);
+        return arg < inputFields.size()
+            ? inputFields.get(arg).getType()
+            : extraInputs.get(arg - inputFields.size()).getType();
+      }
+      public int size() {
+        return argList.size();
+      }
+    };
+  }
+}
+
+// End EnumUtils.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableAggregate.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableAggregate.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableAggregate.java
new file mode 100644
index 0000000..7f19cdf
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableAggregate.java
@@ -0,0 +1,425 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.adapter.enumerable.impl.AggAddContextImpl;
+import org.apache.calcite.adapter.enumerable.impl.AggResultContextImpl;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.linq4j.function.Function0;
+import org.apache.calcite.linq4j.function.Function1;
+import org.apache.calcite.linq4j.function.Function2;
+import org.apache.calcite.linq4j.tree.BlockBuilder;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.linq4j.tree.ParameterExpression;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.prepare.CalcitePrepareImpl;
+import org.apache.calcite.rel.InvalidRelException;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Aggregate;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.SqlAggFunction;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.BuiltInMethod;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
+
+import com.google.common.collect.ImmutableList;
+
+import java.lang.reflect.Type;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.Collections;
+import java.util.List;
+
+/** Implementation of {@link org.apache.calcite.rel.core.Aggregate} in
+ * {@link org.apache.calcite.adapter.enumerable.EnumerableConvention enumerable calling convention}. */
+public class EnumerableAggregate extends Aggregate
+    implements EnumerableRel {
+  private static final List<SqlAggFunction> SUPPORTED_AGGREGATIONS =
+      ImmutableList.of(
+          SqlStdOperatorTable.COUNT,
+          SqlStdOperatorTable.MIN,
+          SqlStdOperatorTable.MAX,
+          SqlStdOperatorTable.SUM);
+
+  public EnumerableAggregate(
+      RelOptCluster cluster,
+      RelTraitSet traitSet,
+      RelNode child,
+      BitSet groupSet,
+      List<AggregateCall> aggCalls)
+      throws InvalidRelException {
+    super(cluster, traitSet, child, groupSet, aggCalls);
+    assert getConvention() instanceof EnumerableConvention;
+
+    for (AggregateCall aggCall : aggCalls) {
+      if (aggCall.isDistinct()) {
+        throw new InvalidRelException(
+            "distinct aggregation not supported");
+      }
+      AggImplementor implementor2 =
+          RexImpTable.INSTANCE.get(aggCall.getAggregation(), false);
+      if (implementor2 == null) {
+        throw new InvalidRelException(
+            "aggregation " + aggCall.getAggregation() + " not supported");
+      }
+    }
+  }
+
+  @Override public EnumerableAggregate copy(RelTraitSet traitSet,
+      RelNode input, BitSet groupSet, List<AggregateCall> aggCalls) {
+    try {
+      return new EnumerableAggregate(getCluster(), traitSet, input,
+          groupSet, aggCalls);
+    } catch (InvalidRelException e) {
+      // Semantic error not possible. Must be a bug. Convert to
+      // internal error.
+      throw new AssertionError(e);
+    }
+  }
+
+  public Result implement(EnumerableRelImplementor implementor, Prefer pref) {
+    final JavaTypeFactory typeFactory = implementor.getTypeFactory();
+    final BlockBuilder builder = new BlockBuilder();
+    final EnumerableRel child = (EnumerableRel) getInput();
+    final Result result = implementor.visitChild(this, 0, child, pref);
+    Expression childExp =
+        builder.append(
+            "child",
+            result.block);
+    final RelDataType inputRowType = getInput().getRowType();
+
+    final PhysType physType =
+        PhysTypeImpl.of(
+            typeFactory, getRowType(), pref.preferCustom());
+
+    // final Enumerable<Employee> child = <<child adapter>>;
+    // Function1<Employee, Integer> keySelector =
+    //     new Function1<Employee, Integer>() {
+    //         public Integer apply(Employee a0) {
+    //             return a0.deptno;
+    //         }
+    //     };
+    // Function1<Employee, Object[]> accumulatorInitializer =
+    //     new Function1<Employee, Object[]>() {
+    //         public Object[] apply(Employee a0) {
+    //             return new Object[] {0, 0};
+    //         }
+    //     };
+    // Function2<Object[], Employee, Object[]> accumulatorAdder =
+    //     new Function2<Object[], Employee, Object[]>() {
+    //         public Object[] apply(Object[] a1, Employee a0) {
+    //              a1[0] = ((Integer) a1[0]) + 1;
+    //              a1[1] = ((Integer) a1[1]) + a0.salary;
+    //             return a1;
+    //         }
+    //     };
+    // Function2<Integer, Object[], Object[]> resultSelector =
+    //     new Function2<Integer, Object[], Object[]>() {
+    //         public Object[] apply(Integer a0, Object[] a1) {
+    //             return new Object[] { a0, a1[0], a1[1] };
+    //         }
+    //     };
+    // return childEnumerable
+    //     .groupBy(
+    //        keySelector, accumulatorInitializer, accumulatorAdder,
+    //        resultSelector);
+    //
+    // or, if key has 0 columns,
+    //
+    // return childEnumerable
+    //     .aggregate(
+    //       accumulatorInitializer.apply(),
+    //       accumulatorAdder,
+    //       resultSelector);
+    //
+    // with a slightly different resultSelector; or if there are no aggregate
+    // functions
+    //
+    // final Enumerable<Employee> child = <<child adapter>>;
+    // Function1<Employee, Integer> keySelector =
+    //     new Function1<Employee, Integer>() {
+    //         public Integer apply(Employee a0) {
+    //             return a0.deptno;
+    //         }
+    //     };
+    // EqualityComparer<Employee> equalityComparer =
+    //     new EqualityComparer<Employee>() {
+    //         boolean equal(Employee a0, Employee a1) {
+    //             return a0.deptno;
+    //         }
+    //     };
+    // return child
+    //     .distinct(equalityComparer);
+
+    final PhysType inputPhysType = result.physType;
+
+    ParameterExpression parameter =
+        Expressions.parameter(inputPhysType.getJavaRowType(), "a0");
+
+    final PhysType keyPhysType =
+        inputPhysType.project(
+            BitSets.toList(groupSet), JavaRowFormat.LIST);
+    final int keyArity = groupSet.cardinality();
+    final Expression keySelector =
+        builder.append(
+            "keySelector",
+            inputPhysType.generateSelector(
+                parameter,
+                BitSets.toList(groupSet),
+                keyPhysType.getFormat()));
+
+    final List<AggImpState> aggs =
+        new ArrayList<AggImpState>(aggCalls.size());
+
+    for (int i = 0; i < aggCalls.size(); i++) {
+      AggregateCall call = aggCalls.get(i);
+      aggs.add(new AggImpState(i, call, false));
+    }
+
+    // Function0<Object[]> accumulatorInitializer =
+    //     new Function0<Object[]>() {
+    //         public Object[] apply() {
+    //             return new Object[] {0, 0};
+    //         }
+    //     };
+    final List<Expression> initExpressions =
+        new ArrayList<Expression>();
+    final BlockBuilder initBlock = new BlockBuilder();
+
+    final List<Type> aggStateTypes = new ArrayList<Type>();
+    for (final AggImpState agg : aggs) {
+      agg.context =
+          new AggContext() {
+            public org.apache.calcite.sql.SqlAggFunction aggregation() {
+              return agg.call.getAggregation();
+            }
+
+            public RelDataType returnRelType() {
+              return agg.call.type;
+            }
+
+            public Type returnType() {
+              return EnumUtils.javaClass(typeFactory, returnRelType());
+            }
+
+            public List<? extends RelDataType> parameterRelTypes() {
+              return EnumUtils.fieldRowTypes(inputRowType, null,
+                  agg.call.getArgList());
+            }
+
+            public List<? extends Type> parameterTypes() {
+              return EnumUtils.fieldTypes(typeFactory,
+                  parameterRelTypes());
+            }
+          };
+      List<Type> state =
+          agg.implementor.getStateType(agg.context);
+
+      if (state.isEmpty()) {
+        continue;
+      }
+
+      aggStateTypes.addAll(state);
+
+      final List<Expression> decls =
+          new ArrayList<Expression>(state.size());
+      for (int i = 0; i < state.size(); i++) {
+        String aggName = "a" + agg.aggIdx;
+        if (CalcitePrepareImpl.DEBUG) {
+          aggName = Util.toJavaId(agg.call.getAggregation().getName(), 0)
+              .substring("ID$0$".length()) + aggName;
+        }
+        Type type = state.get(i);
+        ParameterExpression pe =
+            Expressions.parameter(type,
+                initBlock.newName(aggName + "s" + i));
+        initBlock.add(Expressions.declare(0, pe, null));
+        decls.add(pe);
+      }
+      agg.state = decls;
+      initExpressions.addAll(decls);
+      agg.implementor.implementReset(agg.context,
+          new AggResultContextImpl(initBlock, decls));
+    }
+
+    final PhysType accPhysType =
+        PhysTypeImpl.of(
+            typeFactory,
+            typeFactory.createSyntheticType(aggStateTypes));
+
+    initBlock.add(accPhysType.record(initExpressions));
+
+    final Expression accumulatorInitializer =
+        builder.append(
+            "accumulatorInitializer",
+            Expressions.lambda(
+                Function0.class,
+                initBlock.toBlock()));
+
+    // Function2<Object[], Employee, Object[]> accumulatorAdder =
+    //     new Function2<Object[], Employee, Object[]>() {
+    //         public Object[] apply(Object[] acc, Employee in) {
+    //              acc[0] = ((Integer) acc[0]) + 1;
+    //              acc[1] = ((Integer) acc[1]) + in.salary;
+    //             return acc;
+    //         }
+    //     };
+    final BlockBuilder builder2 = new BlockBuilder();
+    final ParameterExpression inParameter =
+        Expressions.parameter(inputPhysType.getJavaRowType(), "in");
+    final ParameterExpression acc_ =
+        Expressions.parameter(accPhysType.getJavaRowType(), "acc");
+    for (int i = 0, stateOffset = 0; i < aggs.size(); i++) {
+      final AggImpState agg = aggs.get(i);
+
+      int stateSize = agg.state.size();
+      List<Expression> accumulator =
+          new ArrayList<Expression>(stateSize);
+      for (int j = 0; j < stateSize; j++) {
+        accumulator.add(accPhysType.fieldReference(
+            acc_, j + stateOffset));
+      }
+      agg.state = accumulator;
+
+      stateOffset += stateSize;
+
+      AggAddContext addContext =
+          new AggAddContextImpl(builder2, accumulator) {
+            public List<RexNode> rexArguments() {
+              List<RelDataTypeField> inputTypes =
+                  inputPhysType.getRowType().getFieldList();
+              List<RexNode> args = new ArrayList<RexNode>();
+              for (Integer index : agg.call.getArgList()) {
+                args.add(new RexInputRef(index,
+                    inputTypes.get(index).getType()));
+              }
+              return args;
+            }
+
+            public RexToLixTranslator rowTranslator() {
+              return RexToLixTranslator.forAggregation(typeFactory,
+                  currentBlock(), new RexToLixTranslator.InputGetterImpl(
+                      Collections.singletonList(Pair.of(
+                          (Expression) inParameter, inputPhysType))))
+                  .setNullable(currentNullables());
+            }
+          };
+
+      agg.implementor.implementAdd(agg.context, addContext);
+    }
+    builder2.add(acc_);
+    final Expression accumulatorAdder =
+        builder.append(
+            "accumulatorAdder",
+            Expressions.lambda(
+                Function2.class,
+                builder2.toBlock(),
+                acc_,
+                inParameter));
+
+    // Function2<Integer, Object[], Object[]> resultSelector =
+    //     new Function2<Integer, Object[], Object[]>() {
+    //         public Object[] apply(Integer key, Object[] acc) {
+    //             return new Object[] { key, acc[0], acc[1] };
+    //         }
+    //     };
+    final BlockBuilder resultBlock = new BlockBuilder();
+    final List<Expression> results = Expressions.list();
+    final ParameterExpression key_;
+    if (keyArity == 0) {
+      key_ = null;
+    } else {
+      final Type keyType = keyPhysType.getJavaRowType();
+      key_ = Expressions.parameter(keyType, "key");
+      for (int j = 0; j < keyArity; j++) {
+        results.add(
+            keyPhysType.fieldReference(key_, j));
+      }
+    }
+    for (final AggImpState agg : aggs) {
+      results.add(
+          agg.implementor.implementResult(agg.context,
+              new AggResultContextImpl(resultBlock, agg.state)));
+    }
+    resultBlock.add(physType.record(results));
+    if (keyArity == 0) {
+      final Expression resultSelector =
+          builder.append(
+              "resultSelector",
+              Expressions.lambda(
+                  Function1.class,
+                  resultBlock.toBlock(),
+                  acc_));
+      builder.add(
+          Expressions.return_(
+              null,
+              Expressions.call(
+                  BuiltInMethod.SINGLETON_ENUMERABLE.method,
+                  Expressions.call(
+                      childExp,
+                      BuiltInMethod.AGGREGATE.method,
+                      Expressions.call(accumulatorInitializer, "apply"),
+                      accumulatorAdder,
+                      resultSelector))));
+    } else if (aggCalls.isEmpty()
+        && groupSet.equals(
+            BitSets.range(child.getRowType().getFieldCount()))) {
+      builder.add(
+          Expressions.return_(
+              null,
+              Expressions.call(
+                  inputPhysType.convertTo(childExp, physType),
+                  BuiltInMethod.DISTINCT.method,
+                  Expressions.<Expression>list()
+                      .appendIfNotNull(physType.comparer()))));
+    } else {
+      final Expression resultSelector =
+          builder.append(
+              "resultSelector",
+              Expressions.lambda(
+                  Function2.class,
+                  resultBlock.toBlock(),
+                  key_,
+                  acc_));
+      builder.add(
+          Expressions.return_(
+              null,
+              Expressions.call(
+                  childExp,
+                  BuiltInMethod.GROUP_BY2.method,
+                  Expressions
+                      .list(
+                          keySelector,
+                          accumulatorInitializer,
+                          accumulatorAdder,
+                          resultSelector)
+                      .appendIfNotNull(
+                          keyPhysType.comparer()))));
+    }
+    return implementor.result(physType, builder.toBlock());
+  }
+}
+
+// End EnumerableAggregate.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableAggregateRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableAggregateRule.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableAggregateRule.java
new file mode 100644
index 0000000..c04cd03
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableAggregateRule.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.InvalidRelException;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.logical.LogicalAggregate;
+
+/**
+ * Rule to convert a {@link org.apache.calcite.rel.logical.LogicalAggregate}
+ * to an {@link EnumerableAggregate}.
+ */
+class EnumerableAggregateRule extends ConverterRule {
+  EnumerableAggregateRule() {
+    super(LogicalAggregate.class, Convention.NONE,
+        EnumerableConvention.INSTANCE, "EnumerableAggregateRule");
+  }
+
+  public RelNode convert(RelNode rel) {
+    final LogicalAggregate agg = (LogicalAggregate) rel;
+    final RelTraitSet traitSet =
+        agg.getTraitSet().replace(EnumerableConvention.INSTANCE);
+    try {
+      return new EnumerableAggregate(
+          rel.getCluster(),
+          traitSet,
+          convert(agg.getInput(), traitSet),
+          agg.getGroupSet(),
+          agg.getAggCallList());
+    } catch (InvalidRelException e) {
+      EnumerableRules.LOGGER.fine(e.toString());
+      return null;
+    }
+  }
+}
+
+// End EnumerableAggregateRule.java


[33/58] [abbrv] [partial] incubator-calcite git commit: [CALCITE-306] Standardize code style for "import package.*; "

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/rules/FilterProjectTransposeRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/FilterProjectTransposeRule.java b/core/src/main/java/org/apache/calcite/rel/rules/FilterProjectTransposeRule.java
index 2e07d23..2f5b720 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/FilterProjectTransposeRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/FilterProjectTransposeRule.java
@@ -14,27 +14,34 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
+package org.apache.calcite.rel.rules;
 
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.rex.*;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.RelFactories;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexOver;
 
 /**
- * PushFilterPastProjectRule implements the rule for pushing a {@link FilterRel}
- * past a {@link ProjectRel}.
+ * Planner rule that pushes
+ * a {@link org.apache.calcite.rel.logical.LogicalFilter}
+ * past a {@link org.apache.calcite.rel.logical.LogicalProject}.
  */
-public class PushFilterPastProjectRule extends RelOptRule {
+public class FilterProjectTransposeRule extends RelOptRule {
   /** The default instance of
-   * {@link org.eigenbase.rel.rules.PushFilterPastJoinRule}.
+   * {@link org.apache.calcite.rel.rules.FilterProjectTransposeRule}.
    *
    * <p>It matches any kind of join or filter, and generates the same kind of
    * join and filter. It uses null values for {@code filterFactory} and
    * {@code projectFactory} to achieve this. */
-  public static final PushFilterPastProjectRule INSTANCE =
-      new PushFilterPastProjectRule(
-          FilterRelBase.class, null,
-          ProjectRelBase.class, null);
+  public static final FilterProjectTransposeRule INSTANCE =
+      new FilterProjectTransposeRule(
+          Filter.class, null,
+          Project.class, null);
 
   private final RelFactories.FilterFactory filterFactory;
   private final RelFactories.ProjectFactory projectFactory;
@@ -42,15 +49,15 @@ public class PushFilterPastProjectRule extends RelOptRule {
   //~ Constructors -----------------------------------------------------------
 
   /**
-   * Creates a PushFilterPastProjectRule.
+   * Creates a FilterProjectTransposeRule.
    *
    * <p>If {@code filterFactory} is null, creates the same kind of filter as
    * matched in the rule. Similarly {@code projectFactory}.</p>
    */
-  public PushFilterPastProjectRule(
-      Class<? extends FilterRelBase> filterClass,
+  public FilterProjectTransposeRule(
+      Class<? extends Filter> filterClass,
       RelFactories.FilterFactory filterFactory,
-      Class<? extends ProjectRelBase> projectClass,
+      Class<? extends Project> projectClass,
       RelFactories.ProjectFactory projectFactory) {
     super(
         operand(filterClass,
@@ -63,8 +70,8 @@ public class PushFilterPastProjectRule extends RelOptRule {
 
   // implement RelOptRule
   public void onMatch(RelOptRuleCall call) {
-    final FilterRelBase filterRel = call.rel(0);
-    final ProjectRelBase projRel = call.rel(1);
+    final Filter filterRel = call.rel(0);
+    final Project projRel = call.rel(1);
 
     if (RexOver.containsOver(projRel.getProjects(), null)) {
       // In general a filter cannot be pushed below a windowing calculation.
@@ -82,9 +89,9 @@ public class PushFilterPastProjectRule extends RelOptRule {
 
     RelNode newFilterRel =
         filterFactory == null
-            ? filterRel.copy(filterRel.getTraitSet(), projRel.getChild(),
+            ? filterRel.copy(filterRel.getTraitSet(), projRel.getInput(),
                 newCondition)
-            : filterFactory.createFilter(projRel.getChild(), newCondition);
+            : filterFactory.createFilter(projRel.getInput(), newCondition);
 
     RelNode newProjRel =
         projectFactory == null
@@ -97,4 +104,4 @@ public class PushFilterPastProjectRule extends RelOptRule {
   }
 }
 
-// End PushFilterPastProjectRule.java
+// End FilterProjectTransposeRule.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/rules/FilterRemoveIsNotDistinctFromRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/FilterRemoveIsNotDistinctFromRule.java b/core/src/main/java/org/apache/calcite/rel/rules/FilterRemoveIsNotDistinctFromRule.java
index d415f2d..85fe7e2 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/FilterRemoveIsNotDistinctFromRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/FilterRemoveIsNotDistinctFromRule.java
@@ -14,35 +14,45 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.rex.*;
-import org.eigenbase.sql.fun.*;
+package org.apache.calcite.rel.rules;
+
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.logical.LogicalFilter;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexShuttle;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 
 /**
- * Rule to replace isNotDistinctFromOperator with logical equivalent conditions
- * in a {@link FilterRel}.
+ * Planner rule that replaces {@code IS NOT DISTINCT FROM}
+ * in a {@link org.apache.calcite.rel.logical.LogicalFilter}
+ * with logically equivalent operations.
+ *
+ * @see org.apache.calcite.sql.fun.SqlStdOperatorTable#IS_NOT_DISTINCT_FROM
  */
-public final class RemoveIsNotDistinctFromRule extends RelOptRule {
+public final class FilterRemoveIsNotDistinctFromRule extends RelOptRule {
   //~ Static fields/initializers ---------------------------------------------
 
   /** The singleton. */
-  public static final RemoveIsNotDistinctFromRule INSTANCE =
-      new RemoveIsNotDistinctFromRule();
+  public static final FilterRemoveIsNotDistinctFromRule INSTANCE =
+      new FilterRemoveIsNotDistinctFromRule();
 
   //~ Constructors -----------------------------------------------------------
 
-  private RemoveIsNotDistinctFromRule() {
-    super(operand(FilterRel.class, any()));
+  private FilterRemoveIsNotDistinctFromRule() {
+    super(operand(LogicalFilter.class, any()));
   }
 
   //~ Methods ----------------------------------------------------------------
 
   public void onMatch(RelOptRuleCall call) {
-    FilterRel oldFilterRel = call.rel(0);
-    RexNode oldFilterCond = oldFilterRel.getCondition();
+    LogicalFilter oldFilter = call.rel(0);
+    RexNode oldFilterCond = oldFilter.getCondition();
 
     if (RexUtil.findOperatorCall(
         SqlStdOperatorTable.IS_NOT_DISTINCT_FROM,
@@ -57,11 +67,11 @@ public final class RemoveIsNotDistinctFromRule extends RelOptRule {
 
     RemoveIsNotDistinctFromRexShuttle rewriteShuttle =
         new RemoveIsNotDistinctFromRexShuttle(
-            oldFilterRel.getCluster().getRexBuilder());
+            oldFilter.getCluster().getRexBuilder());
 
     RelNode newFilterRel =
         RelOptUtil.createFilter(
-            oldFilterRel.getChild(),
+            oldFilter.getInput(),
             oldFilterCond.accept(rewriteShuttle));
 
     call.transformTo(newFilterRel);
@@ -99,4 +109,4 @@ public final class RemoveIsNotDistinctFromRule extends RelOptRule {
   }
 }
 
-// End RemoveIsNotDistinctFromRule.java
+// End FilterRemoveIsNotDistinctFromRule.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/rules/FilterSetOpTransposeRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/FilterSetOpTransposeRule.java b/core/src/main/java/org/apache/calcite/rel/rules/FilterSetOpTransposeRule.java
index b48fd3f..4e04d0b 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/FilterSetOpTransposeRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/FilterSetOpTransposeRule.java
@@ -14,35 +14,41 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
+package org.apache.calcite.rel.rules;
+
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.core.RelFactories;
+import org.apache.calcite.rel.core.SetOp;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexNode;
 
 import java.util.ArrayList;
 import java.util.List;
 
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.rex.*;
-
 /**
- * PushFilterPastSetOpRule implements the rule for pushing a {@link FilterRel}
- * past a {@link SetOpRel}.
+ * Planner rule that pushes a {@link org.apache.calcite.rel.core.Filter}
+ * past a {@link org.apache.calcite.rel.core.SetOp}.
  */
-public class PushFilterPastSetOpRule extends RelOptRule {
-  public static final PushFilterPastSetOpRule INSTANCE =
-      new PushFilterPastSetOpRule(RelFactories.DEFAULT_FILTER_FACTORY);
+public class FilterSetOpTransposeRule extends RelOptRule {
+  public static final FilterSetOpTransposeRule INSTANCE =
+      new FilterSetOpTransposeRule(RelFactories.DEFAULT_FILTER_FACTORY);
 
   private final RelFactories.FilterFactory filterFactory;
 
   //~ Constructors -----------------------------------------------------------
 
   /**
-   * Creates a PushFilterPastSetOpRule.
+   * Creates a FilterSetOpTransposeRule.
    */
-  public PushFilterPastSetOpRule(RelFactories.FilterFactory filterFactory) {
+  public FilterSetOpTransposeRule(RelFactories.FilterFactory filterFactory) {
     super(
-        operand(FilterRelBase.class,
-            operand(SetOpRel.class, any())));
+        operand(Filter.class,
+            operand(SetOp.class, any())));
     this.filterFactory = filterFactory;
   }
 
@@ -50,8 +56,8 @@ public class PushFilterPastSetOpRule extends RelOptRule {
 
   // implement RelOptRule
   public void onMatch(RelOptRuleCall call) {
-    FilterRelBase filterRel = call.rel(0);
-    SetOpRel setOpRel = call.rel(1);
+    Filter filterRel = call.rel(0);
+    SetOp setOp = call.rel(1);
 
     RexNode condition = filterRel.getCondition();
 
@@ -59,10 +65,10 @@ public class PushFilterPastSetOpRule extends RelOptRule {
     // condition to reference each setop child
     RexBuilder rexBuilder = filterRel.getCluster().getRexBuilder();
     List<RelDataTypeField> origFields =
-        setOpRel.getRowType().getFieldList();
+        setOp.getRowType().getFieldList();
     int[] adjustments = new int[origFields.size()];
     List<RelNode> newSetOpInputs = new ArrayList<RelNode>();
-    for (RelNode input : setOpRel.getInputs()) {
+    for (RelNode input : setOp.getInputs()) {
       RexNode newCondition =
           condition.accept(
               new RelOptUtil.RexInputConverter(
@@ -74,11 +80,11 @@ public class PushFilterPastSetOpRule extends RelOptRule {
     }
 
     // create a new setop whose children are the filters created above
-    SetOpRel newSetOpRel =
-        setOpRel.copy(setOpRel.getTraitSet(), newSetOpInputs);
+    SetOp newSetOp =
+        setOp.copy(setOp.getTraitSet(), newSetOpInputs);
 
-    call.transformTo(newSetOpRel);
+    call.transformTo(newSetOp);
   }
 }
 
-// End PushFilterPastSetOpRule.java
+// End FilterSetOpTransposeRule.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/rules/FilterTableFunctionTransposeRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/FilterTableFunctionTransposeRule.java b/core/src/main/java/org/apache/calcite/rel/rules/FilterTableFunctionTransposeRule.java
index f9f4484..24f6a04 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/FilterTableFunctionTransposeRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/FilterTableFunctionTransposeRule.java
@@ -14,45 +14,53 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
+package org.apache.calcite.rel.rules;
 
-import java.util.*;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.logical.LogicalFilter;
+import org.apache.calcite.rel.logical.LogicalTableFunctionScan;
+import org.apache.calcite.rel.metadata.RelColumnMapping;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexNode;
 
-import org.eigenbase.rel.*;
-import org.eigenbase.rel.metadata.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.rex.*;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
 
 /**
- * PushFilterPastTableFunctionRule implements the rule for pushing a
- * {@link FilterRel} past a {@link TableFunctionRel}.
+ * Planner rule that pushes
+ * a {@link org.apache.calcite.rel.logical.LogicalFilter}
+ * past a {@link org.apache.calcite.rel.logical.LogicalTableFunctionScan}.
  */
-public class PushFilterPastTableFunctionRule extends RelOptRule {
-  public static final PushFilterPastTableFunctionRule INSTANCE =
-      new PushFilterPastTableFunctionRule();
+public class FilterTableFunctionTransposeRule extends RelOptRule {
+  public static final FilterTableFunctionTransposeRule INSTANCE =
+      new FilterTableFunctionTransposeRule();
 
   //~ Constructors -----------------------------------------------------------
 
   /**
-   * Creates a PushFilterPastTableFunctionRule.
+   * Creates a FilterTableFunctionTransposeRule.
    */
-  private PushFilterPastTableFunctionRule() {
+  private FilterTableFunctionTransposeRule() {
     super(
-        operand(
-            FilterRel.class,
-            operand(TableFunctionRel.class, any())));
+        operand(LogicalFilter.class,
+            operand(LogicalTableFunctionScan.class, any())));
   }
 
   //~ Methods ----------------------------------------------------------------
 
   // implement RelOptRule
   public void onMatch(RelOptRuleCall call) {
-    FilterRel filterRel = call.rel(0);
-    TableFunctionRel funcRel = call.rel(1);
+    LogicalFilter filter = call.rel(0);
+    LogicalTableFunctionScan funcRel = call.rel(1);
     Set<RelColumnMapping> columnMappings = funcRel.getColumnMappings();
-    if ((columnMappings == null) || (columnMappings.isEmpty())) {
-      // No column mapping information, so no pushdown
+    if (columnMappings == null || columnMappings.isEmpty()) {
+      // No column mapping information, so no push-down
       // possible.
       return;
     }
@@ -78,11 +86,11 @@ public class PushFilterPastTableFunctionRule extends RelOptRule {
     }
     final List<RelNode> newFuncInputs = new ArrayList<RelNode>();
     final RelOptCluster cluster = funcRel.getCluster();
-    final RexNode condition = filterRel.getCondition();
+    final RexNode condition = filter.getCondition();
 
     // create filters on top of each func input, modifying the filter
     // condition to reference the child instead
-    RexBuilder rexBuilder = filterRel.getCluster().getRexBuilder();
+    RexBuilder rexBuilder = filter.getCluster().getRexBuilder();
     List<RelDataTypeField> origFields = funcRel.getRowType().getFieldList();
     // TODO:  these need to be non-zero once we
     // support arbitrary mappings
@@ -96,12 +104,12 @@ public class PushFilterPastTableFunctionRule extends RelOptRule {
                   funcInput.getRowType().getFieldList(),
                   adjustments));
       newFuncInputs.add(
-          new FilterRel(cluster, funcInput, newCondition));
+          new LogicalFilter(cluster, funcInput, newCondition));
     }
 
     // create a new UDX whose children are the filters created above
-    TableFunctionRel newFuncRel =
-        new TableFunctionRel(
+    LogicalTableFunctionScan newFuncRel =
+        new LogicalTableFunctionScan(
             cluster,
             newFuncInputs,
             funcRel.getCall(),
@@ -111,4 +119,4 @@ public class PushFilterPastTableFunctionRule extends RelOptRule {
   }
 }
 
-// End PushFilterPastTableFunctionRule.java
+// End FilterTableFunctionTransposeRule.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/rules/FilterTableRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/FilterTableRule.java b/core/src/main/java/org/apache/calcite/rel/rules/FilterTableRule.java
index c168ef9..f135dca 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/FilterTableRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/FilterTableRule.java
@@ -14,41 +14,39 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
-
-import java.util.List;
-
-import org.eigenbase.rel.FilterRelBase;
-import org.eigenbase.rel.RelNode;
-import org.eigenbase.rel.TableAccessRelBase;
-import org.eigenbase.relopt.RelOptRule;
-import org.eigenbase.relopt.RelOptRuleCall;
-import org.eigenbase.relopt.RelOptTable;
-import org.eigenbase.relopt.RelOptUtil;
-import org.eigenbase.rex.*;
-
-import net.hydromatic.linq4j.Enumerable;
-
-import net.hydromatic.optiq.DataContext;
-import net.hydromatic.optiq.FilterableTable;
-import net.hydromatic.optiq.ProjectableFilterableTable;
-import net.hydromatic.optiq.rules.java.EnumerableRel;
-import net.hydromatic.optiq.rules.java.JavaRules;
+package org.apache.calcite.rel.rules;
+
+import org.apache.calcite.DataContext;
+import org.apache.calcite.adapter.enumerable.EnumerableInterpreter;
+import org.apache.calcite.adapter.enumerable.EnumerableRel;
+import org.apache.calcite.linq4j.Enumerable;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.schema.FilterableTable;
+import org.apache.calcite.schema.ProjectableFilterableTable;
 
 import com.google.common.base.Predicate;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 
-import static org.eigenbase.util.Static.RESOURCE;
+import java.util.List;
+
+import static org.apache.calcite.util.Static.RESOURCE;
 
 /**
  * Planner rule that pushes a filter into a scan of a {@link FilterableTable}
- * or {@link net.hydromatic.optiq.ProjectableFilterableTable}.
+ * or {@link org.apache.calcite.schema.ProjectableFilterableTable}.
  */
 public class FilterTableRule extends RelOptRule {
-  private static final Predicate<TableAccessRelBase> PREDICATE =
-      new Predicate<TableAccessRelBase>() {
-        public boolean apply(TableAccessRelBase scan) {
+  private static final Predicate<TableScan> PREDICATE =
+      new Predicate<TableScan>() {
+        public boolean apply(TableScan scan) {
           // We can only push filters into a FilterableTable or
           // ProjectableFilterableTable.
           final RelOptTable table = scan.getTable();
@@ -64,18 +62,18 @@ public class FilterTableRule extends RelOptRule {
   /** Creates a FilterTableRule. */
   private FilterTableRule() {
     super(
-        operand(FilterRelBase.class,
-            operand(JavaRules.EnumerableInterpreterRel.class,
-                operand(TableAccessRelBase.class, null, PREDICATE, none()))));
+        operand(Filter.class,
+            operand(EnumerableInterpreter.class,
+                operand(TableScan.class, null, PREDICATE, none()))));
   }
 
   //~ Methods ----------------------------------------------------------------
 
   // implement RelOptRule
   public void onMatch(RelOptRuleCall call) {
-    final FilterRelBase filter = call.rel(0);
-    final JavaRules.EnumerableInterpreterRel interpreter = call.rel(1);
-    final TableAccessRelBase scan = call.rel(2);
+    final Filter filter = call.rel(0);
+    final EnumerableInterpreter interpreter = call.rel(1);
+    final TableScan scan = call.rel(2);
     final FilterableTable filterableTable =
         scan.getTable().unwrap(FilterableTable.class);
     final ProjectableFilterableTable projectableFilterableTable =
@@ -95,10 +93,10 @@ public class FilterTableRule extends RelOptRule {
     // It's worth using the ProjectableFilterableTable interface even if it
     // refused all filters.
     final RelNode newFilter =
-        RelOptUtil.createFilter(interpreter.getChild(),
+        RelOptUtil.createFilter(interpreter.getInput(),
             filterSplit.acceptedFilters, EnumerableRel.FILTER_FACTORY);
     final RelNode newInterpreter =
-        new JavaRules.EnumerableInterpreterRel(interpreter.getCluster(),
+        new EnumerableInterpreter(interpreter.getCluster(),
             interpreter.getTraitSet(), newFilter, 0.15d);
     final RelNode residue =
         RelOptUtil.createFilter(newInterpreter, filterSplit.rejectedFilters);

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/rules/FilterToCalcRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/FilterToCalcRule.java b/core/src/main/java/org/apache/calcite/rel/rules/FilterToCalcRule.java
index d3aaeef..6df80d2 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/FilterToCalcRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/FilterToCalcRule.java
@@ -14,23 +14,34 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
+package org.apache.calcite.rel.rules;
 
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.rex.*;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.logical.LogicalCalc;
+import org.apache.calcite.rel.logical.LogicalFilter;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexProgramBuilder;
 
 import com.google.common.collect.ImmutableList;
 
 /**
- * Planner rule which converts a {@link FilterRel} to a {@link CalcRel}.
+ * Planner rule which converts a
+ * {@link org.apache.calcite.rel.logical.LogicalFilter} to a
+ * {@link org.apache.calcite.rel.logical.LogicalCalc}.
  *
- * <p>The rule does <em>NOT</em> fire if the child is a {@link FilterRel} or a
- * {@link ProjectRel} (we assume they they will be converted using {@link
- * FilterToCalcRule} or {@link ProjectToCalcRule}) or a {@link CalcRel}. This
- * {@link FilterRel} will eventually be converted by {@link
- * MergeFilterOntoCalcRule}.
+ * <p>The rule does <em>NOT</em> fire if the child is a
+ * {@link org.apache.calcite.rel.logical.LogicalFilter} or a
+ * {@link org.apache.calcite.rel.logical.LogicalProject} (we assume they they
+ * will be converted using {@link FilterToCalcRule} or
+ * {@link ProjectToCalcRule}) or a
+ * {@link org.apache.calcite.rel.logical.LogicalCalc}. This
+ * {@link org.apache.calcite.rel.logical.LogicalFilter} will eventually be
+ * converted by {@link FilterCalcMergeRule}.
  */
 public class FilterToCalcRule extends RelOptRule {
   //~ Static fields/initializers ---------------------------------------------
@@ -40,14 +51,14 @@ public class FilterToCalcRule extends RelOptRule {
   //~ Constructors -----------------------------------------------------------
 
   private FilterToCalcRule() {
-    super(operand(FilterRel.class, any()));
+    super(operand(LogicalFilter.class, any()));
   }
 
   //~ Methods ----------------------------------------------------------------
 
   public void onMatch(RelOptRuleCall call) {
-    final FilterRel filter = call.rel(0);
-    final RelNode rel = filter.getChild();
+    final LogicalFilter filter = call.rel(0);
+    final RelNode rel = filter.getInput();
 
     // Create a program containing a filter.
     final RexBuilder rexBuilder = filter.getCluster().getRexBuilder();
@@ -58,8 +69,8 @@ public class FilterToCalcRule extends RelOptRule {
     programBuilder.addCondition(filter.getCondition());
     final RexProgram program = programBuilder.getProgram();
 
-    final CalcRel calc =
-        new CalcRel(
+    final LogicalCalc calc =
+        new LogicalCalc(
             filter.getCluster(),
             filter.getTraitSet(),
             rel,

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/rules/JoinAddRedundantSemiJoinRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/JoinAddRedundantSemiJoinRule.java b/core/src/main/java/org/apache/calcite/rel/rules/JoinAddRedundantSemiJoinRule.java
index c0caa65..2940ac4 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/JoinAddRedundantSemiJoinRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/JoinAddRedundantSemiJoinRule.java
@@ -14,36 +14,44 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
+package org.apache.calcite.rel.rules;
 
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.JoinInfo;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.core.SemiJoin;
+import org.apache.calcite.rel.logical.LogicalJoin;
 
 /**
  * Rule to add a semi-join into a join. Transformation is as follows:
  *
- * <p>JoinRel(X, Y) &rarr; JoinRel(SemiJoinRel(X, Y), Y)
+ * <p>LogicalJoin(X, Y) &rarr; LogicalJoin(SemiJoin(X, Y), Y)
  *
  * <p>The constructor is parameterized to allow any sub-class of
- * {@link JoinRelBase}, not just {@link JoinRel}.</p>
+ * {@link org.apache.calcite.rel.core.Join}, not just
+ * {@link org.apache.calcite.rel.logical.LogicalJoin}.
  */
-public class AddRedundantSemiJoinRule extends RelOptRule {
-  public static final AddRedundantSemiJoinRule INSTANCE =
-      new AddRedundantSemiJoinRule(JoinRel.class);
+public class JoinAddRedundantSemiJoinRule extends RelOptRule {
+  public static final JoinAddRedundantSemiJoinRule INSTANCE =
+      new JoinAddRedundantSemiJoinRule(LogicalJoin.class);
 
   //~ Constructors -----------------------------------------------------------
 
   /**
-   * Creates an AddRedundantSemiJoinRule.
+   * Creates an JoinAddRedundantSemiJoinRule.
    */
-  private AddRedundantSemiJoinRule(Class<? extends JoinRelBase> clazz) {
+  private JoinAddRedundantSemiJoinRule(Class<? extends Join> clazz) {
     super(operand(clazz, any()));
   }
 
   //~ Methods ----------------------------------------------------------------
 
   public void onMatch(RelOptRuleCall call) {
-    JoinRelBase origJoinRel = call.rel(0);
+    Join origJoinRel = call.rel(0);
     if (origJoinRel.isSemiJoinDone()) {
       return;
     }
@@ -60,7 +68,7 @@ public class AddRedundantSemiJoinRule extends RelOptRule {
     }
 
     RelNode semiJoin =
-        new SemiJoinRel(
+        new SemiJoin(
             origJoinRel.getCluster(),
             origJoinRel.getCluster().traitSetOf(Convention.NONE),
             origJoinRel.getLeft(),
@@ -82,4 +90,4 @@ public class AddRedundantSemiJoinRule extends RelOptRule {
   }
 }
 
-// End AddRedundantSemiJoinRule.java
+// End JoinAddRedundantSemiJoinRule.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/rules/JoinAssociateRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/JoinAssociateRule.java b/core/src/main/java/org/apache/calcite/rel/rules/JoinAssociateRule.java
index 0c17984..6a38a9a 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/JoinAssociateRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/JoinAssociateRule.java
@@ -14,60 +14,61 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
+package org.apache.calcite.rel.rules;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.volcano.RelSubset;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexPermuteInputsShuttle;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.mapping.Mappings;
+
+import com.google.common.collect.Lists;
 
 import java.util.BitSet;
 import java.util.List;
 
-import org.eigenbase.rel.JoinRelBase;
-import org.eigenbase.rel.JoinRelType;
-import org.eigenbase.rel.RelNode;
-import org.eigenbase.relopt.RelOptCluster;
-import org.eigenbase.relopt.RelOptRule;
-import org.eigenbase.relopt.RelOptRuleCall;
-import org.eigenbase.relopt.volcano.RelSubset;
-import org.eigenbase.rex.RexBuilder;
-import org.eigenbase.rex.RexNode;
-import org.eigenbase.rex.RexPermuteInputsShuttle;
-import org.eigenbase.rex.RexUtil;
-import org.eigenbase.util.mapping.Mappings;
-
-import net.hydromatic.optiq.util.BitSets;
-
-import com.google.common.collect.Lists;
-
 /**
- * Planner rule that changes a join based on the commutativity rule.
+ * Planner rule that changes a join based on the associativity rule.
  *
  * <p>((a JOIN b) JOIN c) &rarr; (a JOIN (b JOIN c))</p>
  *
  * <p>We do not need a rule to convert (a JOIN (b JOIN c)) &rarr;
  * ((a JOIN b) JOIN c) because we have
- * {@link org.eigenbase.rel.rules.SwapJoinRule}.
+ * {@link JoinCommuteRule}.
+ *
+ * @see JoinCommuteRule
  */
-public class CommutativeJoinRule extends RelOptRule {
+public class JoinAssociateRule extends RelOptRule {
   //~ Static fields/initializers ---------------------------------------------
 
   /** The singleton. */
-  public static final CommutativeJoinRule INSTANCE = new CommutativeJoinRule();
+  public static final JoinAssociateRule INSTANCE = new JoinAssociateRule();
 
   //~ Constructors -----------------------------------------------------------
 
   /**
-   * Creates an CommutativeJoinRule.
+   * Creates a JoinAssociateRule.
    */
-  private CommutativeJoinRule() {
+  private JoinAssociateRule() {
     super(
-        operand(JoinRelBase.class,
-            operand(JoinRelBase.class, any()),
+        operand(Join.class,
+            operand(Join.class, any()),
             operand(RelSubset.class, any())));
   }
 
   //~ Methods ----------------------------------------------------------------
 
   public void onMatch(final RelOptRuleCall call) {
-    final JoinRelBase topJoin = call.rel(0);
-    final JoinRelBase bottomJoin = call.rel(1);
+    final Join topJoin = call.rel(0);
+    final Join bottomJoin = call.rel(1);
     final RelNode relA = bottomJoin.getLeft();
     final RelNode relB = bottomJoin.getRight();
     final RelSubset relC = call.rel(2);
@@ -116,8 +117,8 @@ public class CommutativeJoinRule extends RelOptRule {
     // condition can be pushed down if it does not use columns from A.
     final List<RexNode> top = Lists.newArrayList();
     final List<RexNode> bottom = Lists.newArrayList();
-    PushJoinThroughJoinRule.split(topJoin.getCondition(), aBitSet, top, bottom);
-    PushJoinThroughJoinRule.split(bottomJoin.getCondition(), aBitSet, top,
+    JoinPushThroughJoinRule.split(topJoin.getCondition(), aBitSet, top, bottom);
+    JoinPushThroughJoinRule.split(bottomJoin.getCondition(), aBitSet, top,
         bottom);
 
     // Mapping for moving conditions from topJoin or bottomJoin to
@@ -135,7 +136,7 @@ public class CommutativeJoinRule extends RelOptRule {
     RexNode newBottomCondition =
         RexUtil.composeConjunction(rexBuilder, newBottomList, false);
 
-    final JoinRelBase newBottomJoin =
+    final Join newBottomJoin =
         bottomJoin.copy(bottomJoin.getTraitSet(), newBottomCondition, relB,
             relC, JoinRelType.INNER, false);
 
@@ -143,7 +144,7 @@ public class CommutativeJoinRule extends RelOptRule {
     // Field ordinals do not need to be changed.
     RexNode newTopCondition =
         RexUtil.composeConjunction(rexBuilder, top, false);
-    final JoinRelBase newTopJoin =
+    final Join newTopJoin =
         topJoin.copy(topJoin.getTraitSet(), newTopCondition, relA,
             newBottomJoin, JoinRelType.INNER, false);
 
@@ -151,4 +152,4 @@ public class CommutativeJoinRule extends RelOptRule {
   }
 }
 
-// End CommutativeJoinRule.java
+// End JoinAssociateRule.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 5d353e1..b90ea97 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
@@ -14,41 +14,56 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
-
-import java.util.*;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.rel.RelFactories.ProjectFactory;
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.rex.*;
-import org.eigenbase.util.*;
+package org.apache.calcite.rel.rules;
+
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.core.RelFactories;
+import org.apache.calcite.rel.core.RelFactories.ProjectFactory;
+import org.apache.calcite.rel.logical.LogicalJoin;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
 
+import java.util.List;
+
 /**
- * <code>SwapJoinRule</code> permutes the inputs to a join. Outer joins cannot
- * be permuted.
+ * Planner rule that permutes the inputs to a
+ * {@link org.apache.calcite.rel.core.Join}.
+ *
+ * <p>Outer joins cannot be permuted.
+ *
+ * <p>To preserve the order of columns in the output row, the rule adds a
+ * {@link org.apache.calcite.rel.core.Project}.
  */
-public class SwapJoinRule extends RelOptRule {
+public class JoinCommuteRule extends RelOptRule {
   //~ Static fields/initializers ---------------------------------------------
 
   /** The singleton. */
-  public static final SwapJoinRule INSTANCE = new SwapJoinRule();
+  public static final JoinCommuteRule INSTANCE = new JoinCommuteRule();
 
   private final ProjectFactory projectFactory;
 
   //~ Constructors -----------------------------------------------------------
 
   /**
-   * Creates a SwapJoinRule.
+   * Creates a JoinCommuteRule.
    */
-  private SwapJoinRule() {
-    this(JoinRel.class, RelFactories.DEFAULT_PROJECT_FACTORY);
+  private JoinCommuteRule() {
+    this(LogicalJoin.class, RelFactories.DEFAULT_PROJECT_FACTORY);
   }
 
-  public SwapJoinRule(Class<? extends JoinRelBase> clazz,
+  public JoinCommuteRule(Class<? extends Join> clazz,
       ProjectFactory projectFactory) {
     super(operand(clazz, any()));
     this.projectFactory = projectFactory;
@@ -61,7 +76,7 @@ public class SwapJoinRule extends RelOptRule {
    * modify <code>join</code>. Returns null if the join cannot be swapped (for
    * example, because it is an outer join).
    */
-  public static RelNode swap(JoinRelBase join) {
+  public static RelNode swap(Join join) {
     return swap(join, false);
   }
 
@@ -70,7 +85,7 @@ public class SwapJoinRule extends RelOptRule {
    * @param swapOuterJoins whether outer joins should be swapped
    * @return swapped join if swapping possible; else null
    */
-  public static RelNode swap(JoinRelBase join, boolean swapOuterJoins) {
+  public static RelNode swap(Join join, boolean swapOuterJoins) {
     final JoinRelType joinType = join.getJoinType();
     if (!swapOuterJoins && joinType != JoinRelType.INNER) {
       return null;
@@ -88,7 +103,7 @@ public class SwapJoinRule extends RelOptRule {
     // join, and one for the swapped join, and no more.  This
     // doesn't prevent us from seeing any new combinations assuming
     // that the planner tries the desired order (semijoins after swaps).
-    JoinRelBase newJoin =
+    Join newJoin =
         join.copy(join.getTraitSet(), condition, join.getRight(),
             join.getLeft(), joinType.swap(), join.isSemiJoinDone());
     final List<RexNode> exps =
@@ -101,7 +116,7 @@ public class SwapJoinRule extends RelOptRule {
   }
 
   public void onMatch(final RelOptRuleCall call) {
-    JoinRelBase join = call.rel(0);
+    Join join = call.rel(0);
 
     if (!join.getSystemFieldList().isEmpty()) {
       // FIXME Enable this rule for joins with system fields
@@ -115,10 +130,10 @@ public class SwapJoinRule extends RelOptRule {
 
     // The result is either a Project or, if the project is trivial, a
     // raw Join.
-    final JoinRelBase newJoin =
-        swapped instanceof JoinRelBase
-            ? (JoinRelBase) swapped
-            : (JoinRelBase) swapped.getInput(0);
+    final Join newJoin =
+        swapped instanceof Join
+            ? (Join) swapped
+            : (Join) swapped.getInput(0);
 
     call.transformTo(swapped);
 
@@ -187,8 +202,7 @@ public class SwapJoinRule extends RelOptRule {
               rightFields.get(index).getType(),
               index);
         }
-        throw Util.newInternal(
-            "Bad field offset: index="
+        throw Util.newInternal("Bad field offset: index="
             + var.getIndex()
             + ", leftFieldCount=" + leftFields.size()
             + ", rightFieldCount=" + rightFields.size());
@@ -199,4 +213,4 @@ public class SwapJoinRule extends RelOptRule {
   }
 }
 
-// End SwapJoinRule.java
+// End JoinCommuteRule.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/rules/JoinExtractFilterRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/JoinExtractFilterRule.java b/core/src/main/java/org/apache/calcite/rel/rules/JoinExtractFilterRule.java
index 39563d3..724b318 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/JoinExtractFilterRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/JoinExtractFilterRule.java
@@ -14,42 +14,50 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
+package org.apache.calcite.rel.rules;
 
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.logical.LogicalJoin;
 
 /**
- * Rule to convert an {@link JoinRel inner join} to a {@link FilterRel filter}
- * on top of a {@link JoinRel cartesian inner join}.
+ * Rule to convert an
+ * {@link org.apache.calcite.rel.logical.LogicalJoin inner join} to a
+ * {@link org.apache.calcite.rel.logical.LogicalFilter filter} on top of a
+ * {@link org.apache.calcite.rel.logical.LogicalJoin cartesian inner join}.
  *
  * <p>One benefit of this transformation is that after it, the join condition
  * can be combined with conditions and expressions above the join. It also makes
  * the <code>FennelCartesianJoinRule</code> applicable.
  *
  * <p>The constructor is parameterized to allow any sub-class of
- * {@link JoinRelBase}, not just {@link JoinRel}.</p>
+ * {@link org.apache.calcite.rel.core.Join}, not just
+ * {@link org.apache.calcite.rel.logical.LogicalJoin}.</p>
  */
-public final class ExtractJoinFilterRule extends RelOptRule {
+public final class JoinExtractFilterRule extends RelOptRule {
   //~ Static fields/initializers ---------------------------------------------
 
   /** The singleton. */
-  public static final ExtractJoinFilterRule INSTANCE =
-      new ExtractJoinFilterRule(JoinRel.class);
+  public static final JoinExtractFilterRule INSTANCE =
+      new JoinExtractFilterRule(LogicalJoin.class);
 
   //~ Constructors -----------------------------------------------------------
 
   /**
-   * Creates an ExtractJoinFilterRule.
+   * Creates an JoinExtractFilterRule.
    */
-  public ExtractJoinFilterRule(Class<? extends JoinRelBase> clazz) {
+  public JoinExtractFilterRule(Class<? extends Join> clazz) {
     super(operand(clazz, any()));
   }
 
   //~ Methods ----------------------------------------------------------------
 
   public void onMatch(RelOptRuleCall call) {
-    final JoinRelBase join = call.rel(0);
+    final Join join = call.rel(0);
 
     if (join.getJoinType() != JoinRelType.INNER) {
       return;
@@ -64,7 +72,7 @@ public final class ExtractJoinFilterRule extends RelOptRule {
       return;
     }
 
-    // NOTE jvs 14-Mar-2006:  See SwapJoinRule for why we
+    // NOTE jvs 14-Mar-2006:  See JoinCommuteRule for why we
     // preserve attribute semiJoinDone here.
 
     RelNode cartesianJoinRel =
@@ -85,4 +93,4 @@ public final class ExtractJoinFilterRule extends RelOptRule {
   }
 }
 
-// End ExtractJoinFilterRule.java
+// End JoinExtractFilterRule.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/rules/JoinProjectTransposeRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/JoinProjectTransposeRule.java b/core/src/main/java/org/apache/calcite/rel/rules/JoinProjectTransposeRule.java
index 9af4990..b266e68 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/JoinProjectTransposeRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/JoinProjectTransposeRule.java
@@ -14,60 +14,77 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
+package org.apache.calcite.rel.rules;
 
-import java.util.*;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptRuleOperand;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.RelFactories;
+import org.apache.calcite.rel.core.RelFactories.ProjectFactory;
+import org.apache.calcite.rel.logical.LogicalJoin;
+import org.apache.calcite.rel.logical.LogicalProject;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexLocalRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexProgramBuilder;
+import org.apache.calcite.util.Pair;
 
-import org.eigenbase.rel.*;
-import org.eigenbase.rel.RelFactories.ProjectFactory;
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.rex.*;
-import org.eigenbase.util.Pair;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
 
 /**
- * PullUpProjectsAboveJoinRule implements the rule for pulling {@link
- * ProjectRel}s beneath a {@link JoinRelBase} above the {@link JoinRelBase}. Projections
- * are pulled up if the {@link ProjectRel} doesn't originate from a null
- * generating input in an outer join.
+ * Planner rule that matches a
+ * {@link org.apache.calcite.rel.core.Join} one of whose inputs is a
+ * {@link org.apache.calcite.rel.logical.LogicalProject}, and
+ * pulls the project up.
+ *
+ * <p>Projections are pulled up if the
+ * {@link org.apache.calcite.rel.logical.LogicalProject} doesn't originate from
+ * a null generating input in an outer join.
  */
-public class PullUpProjectsAboveJoinRule extends RelOptRule {
+public class JoinProjectTransposeRule extends RelOptRule {
   //~ Static fields/initializers ---------------------------------------------
 
-  public static final PullUpProjectsAboveJoinRule BOTH_PROJECT =
-      new PullUpProjectsAboveJoinRule(
-          operand(
-              JoinRel.class,
-              operand(ProjectRel.class, any()),
-              operand(ProjectRel.class, any())),
-          "PullUpProjectsAboveJoinRule: with two ProjectRel children");
+  public static final JoinProjectTransposeRule BOTH_PROJECT =
+      new JoinProjectTransposeRule(
+          operand(LogicalJoin.class,
+              operand(LogicalProject.class, any()),
+              operand(LogicalProject.class, any())),
+          "JoinProjectTransposeRule: with two LogicalProject children");
 
-  public static final PullUpProjectsAboveJoinRule LEFT_PROJECT =
-      new PullUpProjectsAboveJoinRule(
-          operand(
-              JoinRel.class,
-              some(
-                  operand(ProjectRel.class, any()))),
-          "PullUpProjectsAboveJoinRule: with ProjectRel on left");
+  public static final JoinProjectTransposeRule LEFT_PROJECT =
+      new JoinProjectTransposeRule(
+          operand(LogicalJoin.class,
+              some(operand(LogicalProject.class, any()))),
+          "JoinProjectTransposeRule: with LogicalProject on left");
 
-  public static final PullUpProjectsAboveJoinRule RIGHT_PROJECT =
-      new PullUpProjectsAboveJoinRule(
+  public static final JoinProjectTransposeRule RIGHT_PROJECT =
+      new JoinProjectTransposeRule(
           operand(
-              JoinRel.class,
+              LogicalJoin.class,
               operand(RelNode.class, any()),
-              operand(ProjectRel.class, any())),
-          "PullUpProjectsAboveJoinRule: with ProjectRel on right");
+              operand(LogicalProject.class, any())),
+          "JoinProjectTransposeRule: with LogicalProject on right");
 
   private final ProjectFactory projectFactory;
 
   //~ Constructors -----------------------------------------------------------
-  public PullUpProjectsAboveJoinRule(
+  public JoinProjectTransposeRule(
       RelOptRuleOperand operand,
       String description) {
     this(operand, description, RelFactories.DEFAULT_PROJECT_FACTORY);
   }
 
-  public PullUpProjectsAboveJoinRule(
+  public JoinProjectTransposeRule(
       RelOptRuleOperand operand,
       String description, ProjectFactory pFactory) {
     super(operand, description);
@@ -78,11 +95,11 @@ public class PullUpProjectsAboveJoinRule extends RelOptRule {
 
   // implement RelOptRule
   public void onMatch(RelOptRuleCall call) {
-    JoinRelBase joinRel = call.rel(0);
+    Join joinRel = call.rel(0);
     JoinRelType joinType = joinRel.getJoinType();
 
-    ProjectRelBase leftProj;
-    ProjectRelBase rightProj;
+    Project leftProj;
+    Project rightProj;
     RelNode leftJoinChild;
     RelNode rightJoinChild;
 
@@ -115,7 +132,7 @@ public class PullUpProjectsAboveJoinRule extends RelOptRule {
     // into the bottom RexProgram.  Note that the join type is an inner
     // join because the inputs haven't actually been joined yet.
     RelDataType joinChildrenRowType =
-        JoinRelBase.deriveJoinRowType(
+        Join.deriveJoinRowType(
             leftJoinChild.getRowType(),
             rightJoinChild.getRowType(),
             JoinRelType.INNER,
@@ -182,13 +199,13 @@ public class PullUpProjectsAboveJoinRule extends RelOptRule {
             bottomProgram,
             rexBuilder);
 
-    // expand out the join condition and construct a new JoinRel that
+    // expand out the join condition and construct a new LogicalJoin that
     // directly references the join children without the intervening
     // ProjectRels
     RexNode newCondition =
         mergedProgram.expandLocalRef(
             mergedProgram.getCondition());
-    JoinRelBase newJoinRel =
+    Join newJoinRel =
         joinRel.copy(joinRel.getTraitSet(), newCondition,
             leftJoinChild, rightJoinChild, joinRel.getJoinType(),
             joinRel.isSemiJoinDone());
@@ -227,7 +244,7 @@ public class PullUpProjectsAboveJoinRule extends RelOptRule {
    * @return true if the rule was invoked with a left project child
    */
   protected boolean hasLeftChild(RelOptRuleCall call) {
-    return call.rel(1) instanceof ProjectRelBase;
+    return call.rel(1) instanceof Project;
   }
 
   /**
@@ -240,27 +257,27 @@ public class PullUpProjectsAboveJoinRule extends RelOptRule {
 
   /**
    * @param call RelOptRuleCall
-   * @return ProjectRel corresponding to the right child
+   * @return LogicalProject corresponding to the right child
    */
-  protected ProjectRelBase getRightChild(RelOptRuleCall call) {
+  protected Project getRightChild(RelOptRuleCall call) {
     return call.rel(2);
   }
 
   /**
    * Returns the child of the project that will be used as input into the new
-   * JoinRel once the projects are pulled above the JoinRel.
+   * LogicalJoin once the projects are pulled above the LogicalJoin.
    *
    * @param call      RelOptRuleCall
    * @param project   project RelNode
    * @param leftChild true if the project corresponds to the left projection
    * @return child of the project that will be used as input into the new
-   * JoinRel once the projects are pulled above the JoinRel
+   * LogicalJoin once the projects are pulled above the LogicalJoin
    */
   protected RelNode getProjectChild(
       RelOptRuleCall call,
-      ProjectRelBase project,
+      Project project,
       boolean leftChild) {
-    return project.getChild();
+    return project.getInput();
   }
 
   /**
@@ -278,7 +295,7 @@ public class PullUpProjectsAboveJoinRule extends RelOptRule {
    * @param projects           Projection expressions &amp; names to be created
    */
   private void createProjectExprs(
-      ProjectRelBase projRel,
+      Project projRel,
       RelNode joinChild,
       int adjustmentAmount,
       RexBuilder rexBuilder,
@@ -322,4 +339,4 @@ public class PullUpProjectsAboveJoinRule extends RelOptRule {
   }
 }
 
-// End PullUpProjectsAboveJoinRule.java
+// End JoinProjectTransposeRule.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/rules/JoinPushThroughJoinRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/JoinPushThroughJoinRule.java b/core/src/main/java/org/apache/calcite/rel/rules/JoinPushThroughJoinRule.java
index 76b7f2e..8e024aa 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/JoinPushThroughJoinRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/JoinPushThroughJoinRule.java
@@ -14,17 +14,28 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
-
-import java.util.*;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.rel.RelFactories.ProjectFactory;
-import org.eigenbase.relopt.*;
-import org.eigenbase.rex.*;
-import org.eigenbase.util.mapping.Mappings;
-
-import net.hydromatic.optiq.util.BitSets;
+package org.apache.calcite.rel.rules;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.core.RelFactories;
+import org.apache.calcite.rel.core.RelFactories.ProjectFactory;
+import org.apache.calcite.rel.logical.LogicalJoin;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexPermuteInputsShuttle;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.mapping.Mappings;
+
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.List;
 
 /**
  * Rule that pushes the right input of a join into through the left input of
@@ -49,19 +60,19 @@ import net.hydromatic.optiq.util.BitSets;
  * <p>Before the rule, one join has two conditions and the other has none
  * ({@code ON TRUE}). After the rule, each join has one condition.</p>
  */
-public class PushJoinThroughJoinRule extends RelOptRule {
+public class JoinPushThroughJoinRule extends RelOptRule {
   /** Instance of the rule that works on logical joins only, and pushes to the
    * right. */
   public static final RelOptRule RIGHT =
-      new PushJoinThroughJoinRule(
-          "PushJoinThroughJoinRule:right", true, JoinRel.class,
+      new JoinPushThroughJoinRule(
+          "JoinPushThroughJoinRule:right", true, LogicalJoin.class,
           RelFactories.DEFAULT_PROJECT_FACTORY);
 
   /** Instance of the rule that works on logical joins only, and pushes to the
    * left. */
   public static final RelOptRule LEFT =
-      new PushJoinThroughJoinRule(
-          "PushJoinThroughJoinRule:left", false, JoinRel.class,
+      new JoinPushThroughJoinRule(
+          "JoinPushThroughJoinRule:left", false, LogicalJoin.class,
           RelFactories.DEFAULT_PROJECT_FACTORY);
 
   private final boolean right;
@@ -69,13 +80,12 @@ public class PushJoinThroughJoinRule extends RelOptRule {
   private final ProjectFactory projectFactory;
 
   /**
-   * Creates a PushJoinThroughJoinRule.
+   * Creates a JoinPushThroughJoinRule.
    */
-  public PushJoinThroughJoinRule(String description, boolean right,
-      Class<? extends JoinRelBase> clazz, ProjectFactory projectFactory) {
+  public JoinPushThroughJoinRule(String description, boolean right,
+      Class<? extends Join> clazz, ProjectFactory projectFactory) {
     super(
-        operand(
-            clazz,
+        operand(clazz,
             operand(clazz, any()),
             operand(RelNode.class, any())),
         description);
@@ -83,8 +93,7 @@ public class PushJoinThroughJoinRule extends RelOptRule {
     this.projectFactory = projectFactory;
   }
 
-  @Override
-  public void onMatch(RelOptRuleCall call) {
+  @Override public void onMatch(RelOptRuleCall call) {
     if (right) {
       onMatchRight(call);
     } else {
@@ -93,8 +102,8 @@ public class PushJoinThroughJoinRule extends RelOptRule {
   }
 
   private void onMatchRight(RelOptRuleCall call) {
-    final JoinRelBase topJoin = call.rel(0);
-    final JoinRelBase bottomJoin = call.rel(1);
+    final Join topJoin = call.rel(0);
+    final Join bottomJoin = call.rel(1);
     final RelNode relC = call.rel(2);
     final RelNode relA = bottomJoin.getLeft();
     final RelNode relB = bottomJoin.getRight();
@@ -164,7 +173,7 @@ public class PushJoinThroughJoinRule extends RelOptRule {
     final RexBuilder rexBuilder = cluster.getRexBuilder();
     RexNode newBottomCondition =
         RexUtil.composeConjunction(rexBuilder, newBottomList, false);
-    final JoinRelBase newBottomJoin =
+    final Join newBottomJoin =
         bottomJoin.copy(bottomJoin.getTraitSet(), newBottomCondition, relA,
             relC, bottomJoin.getJoinType(), bottomJoin.isSemiJoinDone());
 
@@ -184,7 +193,7 @@ public class PushJoinThroughJoinRule extends RelOptRule {
     RexNode newTopCondition =
         RexUtil.composeConjunction(rexBuilder, newTopList, false);
     @SuppressWarnings("SuspiciousNameCombination")
-    final JoinRelBase newTopJoin =
+    final Join newTopJoin =
         topJoin.copy(topJoin.getTraitSet(), newTopCondition, newBottomJoin,
             relB, topJoin.getJoinType(), topJoin.isSemiJoinDone());
 
@@ -200,8 +209,8 @@ public class PushJoinThroughJoinRule extends RelOptRule {
    * of the two lower siblings, rather than the right.
    */
   private void onMatchLeft(RelOptRuleCall call) {
-    final JoinRelBase topJoin = call.rel(0);
-    final JoinRelBase bottomJoin = call.rel(1);
+    final Join topJoin = call.rel(0);
+    final Join bottomJoin = call.rel(1);
     final RelNode relC = call.rel(2);
     final RelNode relA = bottomJoin.getLeft();
     final RelNode relB = bottomJoin.getRight();
@@ -272,7 +281,7 @@ public class PushJoinThroughJoinRule extends RelOptRule {
     final RexBuilder rexBuilder = cluster.getRexBuilder();
     RexNode newBottomCondition =
         RexUtil.composeConjunction(rexBuilder, newBottomList, false);
-    final JoinRelBase newBottomJoin =
+    final Join newBottomJoin =
         bottomJoin.copy(bottomJoin.getTraitSet(), newBottomCondition, relC,
             relB, bottomJoin.getJoinType(), bottomJoin.isSemiJoinDone());
 
@@ -292,7 +301,7 @@ public class PushJoinThroughJoinRule extends RelOptRule {
     RexNode newTopCondition =
         RexUtil.composeConjunction(rexBuilder, newTopList, false);
     @SuppressWarnings("SuspiciousNameCombination")
-    final JoinRelBase newTopJoin =
+    final Join newTopJoin =
         topJoin.copy(topJoin.getTraitSet(), newTopCondition, newBottomJoin,
             relA, topJoin.getJoinType(), topJoin.isSemiJoinDone());
 
@@ -322,4 +331,4 @@ public class PushJoinThroughJoinRule extends RelOptRule {
   }
 }
 
-// End PushJoinThroughJoinRule.java
+// End JoinPushThroughJoinRule.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/rules/JoinPushTransitivePredicatesRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/JoinPushTransitivePredicatesRule.java b/core/src/main/java/org/apache/calcite/rel/rules/JoinPushTransitivePredicatesRule.java
index ab08289..3b7056d 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/JoinPushTransitivePredicatesRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/JoinPushTransitivePredicatesRule.java
@@ -14,42 +14,45 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
+package org.apache.calcite.rel.rules;
 
-import org.eigenbase.rel.JoinRelBase;
-import org.eigenbase.rel.RelFactories;
-import org.eigenbase.rel.RelNode;
-import org.eigenbase.rel.metadata.RelMetadataQuery;
-import org.eigenbase.relopt.RelOptPredicateList;
-import org.eigenbase.relopt.RelOptRule;
-import org.eigenbase.relopt.RelOptRuleCall;
-import org.eigenbase.rex.RexBuilder;
-import org.eigenbase.rex.RexUtil;
+import org.apache.calcite.plan.RelOptPredicateList;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.RelFactories;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexUtil;
 
 /**
- * A Rule to apply inferred predicates from
- * {@link org.eigenbase.rel.metadata.RelMdPredicates}.
+ * Planner rule that infers predicates from on a
+ * {@link org.apache.calcite.rel.core.Join} and creates
+ * {@link org.apache.calcite.rel.core.Filter}s if those predicates can be pushed
+ * to its inputs.
  *
- * <p>Predicates returned in {@link org.eigenbase.relopt.RelOptPredicateList}
- * are applied appropriately.
+ * <p>Uses {@link org.apache.calcite.rel.metadata.RelMdPredicates} to infer
+ * the predicates,
+ * returns them in a {@link org.apache.calcite.plan.RelOptPredicateList}
+ * and applies them appropriately.
  */
-public class TransitivePredicatesOnJoinRule extends RelOptRule {
+public class JoinPushTransitivePredicatesRule extends RelOptRule {
   private final RelFactories.FilterFactory filterFactory;
 
   /** The singleton. */
-  public static final TransitivePredicatesOnJoinRule INSTANCE =
-      new TransitivePredicatesOnJoinRule(JoinRelBase.class,
+  public static final JoinPushTransitivePredicatesRule INSTANCE =
+      new JoinPushTransitivePredicatesRule(Join.class,
           RelFactories.DEFAULT_FILTER_FACTORY);
 
-  public TransitivePredicatesOnJoinRule(Class<? extends JoinRelBase> clazz,
+  public JoinPushTransitivePredicatesRule(Class<? extends Join> clazz,
       RelFactories.FilterFactory filterFactory) {
     super(operand(clazz, any()));
     this.filterFactory = filterFactory;
   }
 
-  @Override
-  public void onMatch(RelOptRuleCall call) {
-    JoinRelBase join = call.rel(0);
+  @Override public void onMatch(RelOptRuleCall call) {
+    Join join = call.rel(0);
     RelOptPredicateList preds = RelMetadataQuery.getPulledUpPredicates(join);
 
     if (preds.leftInferredPredicates.isEmpty()
@@ -83,4 +86,4 @@ public class TransitivePredicatesOnJoinRule extends RelOptRule {
   }
 }
 
-// End TransitivePredicatesOnJoinRule.java
+// End JoinPushTransitivePredicatesRule.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/rules/JoinToCorrelatorRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/JoinToCorrelatorRule.java b/core/src/main/java/org/apache/calcite/rel/rules/JoinToCorrelatorRule.java
index de40ede..219b662 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/JoinToCorrelatorRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/JoinToCorrelatorRule.java
@@ -14,21 +14,31 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
+package org.apache.calcite.rel.rules;
 
-import java.util.*;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.rex.*;
-import org.eigenbase.sql.fun.*;
-import org.eigenbase.util.*;
-import org.eigenbase.util.mapping.IntPair;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptQuery;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Correlation;
+import org.apache.calcite.rel.core.Correlator;
+import org.apache.calcite.rel.core.JoinInfo;
+import org.apache.calcite.rel.logical.LogicalJoin;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.util.Util;
+import org.apache.calcite.util.mapping.IntPair;
 
 import com.google.common.collect.Lists;
 
+import java.util.List;
+
 /**
- * Rule which converts a {@link JoinRel} into a {@link CorrelatorRel}, which can
+ * Rule that converts a {@link org.apache.calcite.rel.logical.LogicalJoin}
+ * into a {@link org.apache.calcite.rel.core.Correlator}, which can
  * then be implemented using nested loops.
  *
  * <p>For example,</p>
@@ -36,8 +46,8 @@ import com.google.common.collect.Lists;
  * <blockquote><code>select * from emp join dept on emp.deptno =
  * dept.deptno</code></blockquote>
  *
- * <p>becomes a CorrelatorRel which restarts TableAccessRel("DEPT") for each row
- * read from TableAccessRel("EMP").</p>
+ * <p>becomes a Correlator which restarts LogicalTableScan("DEPT") for each
+ * row read from LogicalTableScan("EMP").</p>
  *
  * <p>This rule is not applicable if for certain types of outer join. For
  * example,</p>
@@ -46,27 +56,27 @@ import com.google.common.collect.Lists;
  * dept.deptno</code></blockquote>
  *
  * <p>would require emitting a NULL emp row if a certain department contained no
- * employees, and CorrelatorRel cannot do that.</p>
+ * employees, and Correlator cannot do that.</p>
  */
-public class NestedLoopsJoinRule extends RelOptRule {
+public class JoinToCorrelatorRule extends RelOptRule {
   //~ Static fields/initializers ---------------------------------------------
 
-  public static final NestedLoopsJoinRule INSTANCE =
-      new NestedLoopsJoinRule();
+  public static final JoinToCorrelatorRule INSTANCE =
+      new JoinToCorrelatorRule();
 
   //~ Constructors -----------------------------------------------------------
 
   /**
    * Private constructor; use singleton {@link #INSTANCE}.
    */
-  private NestedLoopsJoinRule() {
-    super(operand(JoinRel.class, any()));
+  private JoinToCorrelatorRule() {
+    super(operand(LogicalJoin.class, any()));
   }
 
   //~ Methods ----------------------------------------------------------------
 
   public boolean matches(RelOptRuleCall call) {
-    JoinRel join = call.rel(0);
+    LogicalJoin join = call.rel(0);
     switch (join.getJoinType()) {
     case INNER:
     case LEFT:
@@ -81,7 +91,7 @@ public class NestedLoopsJoinRule extends RelOptRule {
 
   public void onMatch(RelOptRuleCall call) {
     assert matches(call);
-    final JoinRel join = call.rel(0);
+    final LogicalJoin join = call.rel(0);
     RelNode right = join.getRight();
     final RelNode left = join.getLeft();
     final JoinInfo joinInfo = join.analyzeCondition();
@@ -105,7 +115,7 @@ public class NestedLoopsJoinRule extends RelOptRule {
     }
     final RelNode filteredRight = RelOptUtil.createFilter(right, conditions);
     RelNode newRel =
-        new CorrelatorRel(
+        new Correlator(
             join.getCluster(),
             left,
             filteredRight,
@@ -116,4 +126,4 @@ public class NestedLoopsJoinRule extends RelOptRule {
   }
 }
 
-// End NestedLoopsJoinRule.java
+// End JoinToCorrelatorRule.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/rules/JoinToMultiJoinRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/JoinToMultiJoinRule.java b/core/src/main/java/org/apache/calcite/rel/rules/JoinToMultiJoinRule.java
index ec003b3..082575a 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/JoinToMultiJoinRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/JoinToMultiJoinRule.java
@@ -14,40 +14,55 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
-
-import java.util.*;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.rex.*;
-import org.eigenbase.util.ImmutableIntList;
-import org.eigenbase.util.Pair;
+package org.apache.calcite.rel.rules;
+
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.logical.LogicalJoin;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.util.ImmutableIntList;
+import org.apache.calcite.util.Pair;
 
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 
+import java.util.BitSet;
+import java.util.List;
+import java.util.Map;
+
 /**
- * Rule to flatten a tree of {@link JoinRel}s into a single {@link MultiJoinRel}
- * with N inputs. An input is not flattened if the input is a null generating
- * input in an outer join, i.e., either input in a full outer join, the right
- * hand side of a left outer join, or the left hand side of a right outer join.
+ * Planner rule to flatten a tree of
+ * {@link org.apache.calcite.rel.logical.LogicalJoin}s
+ * into a single {@link MultiJoin} with N inputs.
+ *
+ * <p>An input is not flattened if
+ * the input is a null generating input in an outer join, i.e., either input in
+ * a full outer join, the right hand side of a left outer join, or the left hand
+ * side of a right outer join.
  *
  * <p>Join conditions are also pulled up from the inputs into the topmost
- * {@link MultiJoinRel},
+ * {@link MultiJoin},
  * unless the input corresponds to a null generating input in an outer join,
  *
- * <p>Outer join information is also stored in the {@link MultiJoinRel}. A
+ * <p>Outer join information is also stored in the {@link MultiJoin}. A
  * boolean flag indicates if the join is a full outer join, and in the case of
  * left and right outer joins, the join type and outer join conditions are
- * stored in arrays in the {@link MultiJoinRel}. This outer join information is
+ * stored in arrays in the {@link MultiJoin}. This outer join information is
  * associated with the null generating input in the outer join. So, in the case
  * of a a left outer join between A and B, the information is associated with B,
  * not A.
  *
- * <p>Here are examples of the {@link MultiJoinRel}s constructed after this rule
+ * <p>Here are examples of the {@link MultiJoin}s constructed after this rule
  * has been applied on following join trees.
  *
  * <ul>
@@ -62,35 +77,39 @@ import com.google.common.collect.Maps;
  * <li>A FULL JOIN B &rarr; MJ[full](A, B)
  *
  * <li>A LEFT JOIN (B JOIN C) &rarr; MJ(A, MJ(B, C))), left outer join on
- * input#1 in the outermost MultiJoinRel
+ * input#1 in the outermost MultiJoin
  *
  * <li>(A JOIN B) LEFT JOIN C &rarr; MJ(A, B, C), left outer join on input#2
  *
  * <li>(A LEFT JOIN B) JOIN C &rarr; MJ(MJ(A, B), C), left outer join on input#1
- * of the inner MultiJoinRel        TODO
+ * of the inner MultiJoin        TODO
  *
  * <li>A LEFT JOIN (B FULL JOIN C) &rarr; MJ(A, MJ[full](B, C)), left outer join
- * on input#1 in the outermost MultiJoinRel
+ * on input#1 in the outermost MultiJoin
  *
  * <li>(A LEFT JOIN B) FULL JOIN (C RIGHT JOIN D) &rarr;
  *      MJ[full](MJ(A, B), MJ(C, D)), left outer join on input #1 in the first
- *      inner MultiJoinRel and right outer join on input#0 in the second inner
- *      MultiJoinRel
+ *      inner MultiJoin and right outer join on input#0 in the second inner
+ *      MultiJoin
  * </ul>
  *
  * <p>The constructor is parameterized to allow any sub-class of
- * {@link JoinRelBase}, not just {@link JoinRel}.</p>
+ * {@link org.apache.calcite.rel.core.Join}, not just
+ * {@link org.apache.calcite.rel.logical.LogicalJoin}.</p>
+ *
+ * @see org.apache.calcite.rel.rules.FilterMultiJoinMergeRule
+ * @see org.apache.calcite.rel.rules.ProjectMultiJoinMergeRule
  */
-public class ConvertMultiJoinRule extends RelOptRule {
-  public static final ConvertMultiJoinRule INSTANCE =
-      new ConvertMultiJoinRule(JoinRelBase.class);
+public class JoinToMultiJoinRule extends RelOptRule {
+  public static final JoinToMultiJoinRule INSTANCE =
+      new JoinToMultiJoinRule(LogicalJoin.class);
 
   //~ Constructors -----------------------------------------------------------
 
   /**
-   * Creates a ConvertMultiJoinRule.
+   * Creates a JoinToMultiJoinRule.
    */
-  public ConvertMultiJoinRule(Class<? extends JoinRelBase> clazz) {
+  public JoinToMultiJoinRule(Class<? extends Join> clazz) {
     super(
         operand(clazz,
             operand(RelNode.class, any()),
@@ -100,12 +119,12 @@ public class ConvertMultiJoinRule extends RelOptRule {
   //~ Methods ----------------------------------------------------------------
 
   public void onMatch(RelOptRuleCall call) {
-    final JoinRelBase origJoin = call.rel(0);
+    final Join origJoin = call.rel(0);
     final RelNode left = call.rel(1);
     final RelNode right = call.rel(2);
 
-    // combine the children MultiJoinRel inputs into an array of inputs
-    // for the new MultiJoinRel
+    // combine the children MultiJoin inputs into an array of inputs
+    // for the new MultiJoin
     final List<BitSet> projFieldsList = Lists.newArrayList();
     final List<int[]> joinFieldRefCountsList = Lists.newArrayList();
     final List<RelNode> newInputs =
@@ -128,12 +147,12 @@ public class ConvertMultiJoinRule extends RelOptRule {
         joinSpecs);
 
     // pull up the join filters from the children MultiJoinRels and
-    // combine them with the join filter associated with this JoinRel to
-    // form the join filter for the new MultiJoinRel
+    // combine them with the join filter associated with this LogicalJoin to
+    // form the join filter for the new MultiJoin
     List<RexNode> newJoinFilters = combineJoinFilters(origJoin, left, right);
 
     // add on the join field reference counts for the join condition
-    // associated with this JoinRel
+    // associated with this LogicalJoin
     final ImmutableMap<Integer, ImmutableIntList> newJoinFieldRefCountsMap =
         addOnJoinFieldRefCounts(newInputs,
             origJoin.getRowType().getFieldCount(),
@@ -145,7 +164,7 @@ public class ConvertMultiJoinRule extends RelOptRule {
 
     final RexBuilder rexBuilder = origJoin.getCluster().getRexBuilder();
     RelNode multiJoin =
-        new MultiJoinRel(
+        new MultiJoin(
             origJoin.getCluster(),
             newInputs,
             RexUtil.composeConjunction(rexBuilder, newJoinFilters, false),
@@ -161,7 +180,7 @@ public class ConvertMultiJoinRule extends RelOptRule {
   }
 
   /**
-   * Combines the inputs into a JoinRel into an array of inputs.
+   * Combines the inputs into a LogicalJoin into an array of inputs.
    *
    * @param join                   original join
    * @param left                   left input into join
@@ -173,7 +192,7 @@ public class ConvertMultiJoinRule extends RelOptRule {
    * @return combined left and right inputs in an array
    */
   private List<RelNode> combineInputs(
-      JoinRelBase join,
+      Join join,
       RelNode left,
       RelNode right,
       List<BitSet> projFieldsList,
@@ -183,7 +202,7 @@ public class ConvertMultiJoinRule extends RelOptRule {
     // leave the null generating sides of an outer join intact; don't
     // pull up those children inputs into the array we're constructing
     if (canCombine(left, join.getJoinType().generatesNullsOnLeft())) {
-      final MultiJoinRel leftMultiJoin = (MultiJoinRel) left;
+      final MultiJoin leftMultiJoin = (MultiJoin) left;
       for (int i = 0; i < left.getInputs().size(); i++) {
         newInputs.add(leftMultiJoin.getInput(i));
         projFieldsList.add(leftMultiJoin.getProjFields().get(i));
@@ -198,7 +217,7 @@ public class ConvertMultiJoinRule extends RelOptRule {
     }
 
     if (canCombine(right, join.getJoinType().generatesNullsOnRight())) {
-      final MultiJoinRel rightMultiJoin = (MultiJoinRel) right;
+      final MultiJoin rightMultiJoin = (MultiJoin) right;
       for (int i = 0; i < right.getInputs().size(); i++) {
         newInputs.add(rightMultiJoin.getInput(i));
         projFieldsList.add(
@@ -231,7 +250,7 @@ public class ConvertMultiJoinRule extends RelOptRule {
    *                       copied
    */
   private void combineOuterJoins(
-      JoinRelBase joinRel,
+      Join joinRel,
       List<RelNode> combinedInputs,
       RelNode left,
       RelNode right,
@@ -245,7 +264,7 @@ public class ConvertMultiJoinRule extends RelOptRule {
     case LEFT:
       if (leftCombined) {
         copyOuterJoinInfo(
-            (MultiJoinRel) left,
+            (MultiJoin) left,
             joinSpecs,
             0,
             null,
@@ -259,7 +278,7 @@ public class ConvertMultiJoinRule extends RelOptRule {
       joinSpecs.add(Pair.of(joinType, joinRel.getCondition()));
       if (rightCombined) {
         copyOuterJoinInfo(
-            (MultiJoinRel) right,
+            (MultiJoin) right,
             joinSpecs,
             left.getRowType().getFieldCount(),
             right.getRowType().getFieldList(),
@@ -271,7 +290,7 @@ public class ConvertMultiJoinRule extends RelOptRule {
     default:
       if (leftCombined) {
         copyOuterJoinInfo(
-            (MultiJoinRel) left,
+            (MultiJoin) left,
             joinSpecs,
             0,
             null,
@@ -281,7 +300,7 @@ public class ConvertMultiJoinRule extends RelOptRule {
       }
       if (rightCombined) {
         copyOuterJoinInfo(
-            (MultiJoinRel) right,
+            (MultiJoin) right,
             joinSpecs,
             left.getRowType().getFieldCount(),
             right.getRowType().getFieldList(),
@@ -293,11 +312,11 @@ public class ConvertMultiJoinRule extends RelOptRule {
   }
 
   /**
-   * Copies outer join data from a source MultiJoinRel to a new set of arrays.
+   * Copies outer join data from a source MultiJoin to a new set of arrays.
    * Also adjusts the conditions to reflect the new position of an input if
    * that input ends up being shifted to the right.
    *
-   * @param multiJoinRel     the source MultiJoinRel
+   * @param multiJoin     the source MultiJoin
    * @param destJoinSpecs    the list where the join types and conditions will
    *                         be copied
    * @param adjustmentAmount if &gt; 0, the amount the RexInputRefs in the join
@@ -307,15 +326,15 @@ public class ConvertMultiJoinRule extends RelOptRule {
    * @param destFields       the destination fields that the new join conditions
    */
   private void copyOuterJoinInfo(
-      MultiJoinRel multiJoinRel,
+      MultiJoin multiJoin,
       List<Pair<JoinRelType, RexNode>> destJoinSpecs,
       int adjustmentAmount,
       List<RelDataTypeField> srcFields,
       List<RelDataTypeField> destFields) {
     final List<Pair<JoinRelType, RexNode>> srcJoinSpecs =
         Pair.zip(
-            multiJoinRel.getJoinTypes(),
-            multiJoinRel.getOuterJoinConditions());
+            multiJoin.getJoinTypes(),
+            multiJoin.getOuterJoinConditions());
 
     if (adjustmentAmount == 0) {
       destJoinSpecs.addAll(srcJoinSpecs);
@@ -334,7 +353,7 @@ public class ConvertMultiJoinRule extends RelOptRule {
                     ? null
                     : src.right.accept(
                         new RelOptUtil.RexInputConverter(
-                            multiJoinRel.getCluster().getRexBuilder(),
+                            multiJoin.getCluster().getRexBuilder(),
                             srcFields, destFields, adjustments))));
       }
     }
@@ -352,7 +371,7 @@ public class ConvertMultiJoinRule extends RelOptRule {
    * @return combined join filters AND-ed together
    */
   private List<RexNode> combineJoinFilters(
-      JoinRelBase joinRel,
+      Join joinRel,
       RelNode left,
       RelNode right) {
     JoinRelType joinType = joinRel.getJoinType();
@@ -365,12 +384,12 @@ public class ConvertMultiJoinRule extends RelOptRule {
       filters.add(joinRel.getCondition());
     }
     if (canCombine(left, joinType.generatesNullsOnLeft())) {
-      filters.add(((MultiJoinRel) left).getJoinFilter());
+      filters.add(((MultiJoin) left).getJoinFilter());
     }
     // Need to adjust the RexInputs of the right child, since
     // those need to shift over to the right
     if (canCombine(right, joinType.generatesNullsOnRight())) {
-      MultiJoinRel multiJoin = (MultiJoinRel) right;
+      MultiJoin multiJoin = (MultiJoin) right;
       filters.add(
           shiftRightFilter(joinRel, left, multiJoin,
               multiJoin.getJoinFilter()));
@@ -385,30 +404,30 @@ public class ConvertMultiJoinRule extends RelOptRule {
    *
    * @param input          input into a join
    * @param nullGenerating true if the input is null generating
-   * @return true if the input can be combined into a parent MultiJoinRel
+   * @return true if the input can be combined into a parent MultiJoin
    */
   private boolean canCombine(RelNode input, boolean nullGenerating) {
-    return input instanceof MultiJoinRel
-        && !((MultiJoinRel) input).isFullOuterJoin()
-        && !((MultiJoinRel) input).containsOuter()
+    return input instanceof MultiJoin
+        && !((MultiJoin) input).isFullOuterJoin()
+        && !((MultiJoin) input).containsOuter()
         && !nullGenerating;
   }
 
   /**
-   * Shifts a filter originating from the right child of the JoinRel to the
+   * Shifts a filter originating from the right child of the LogicalJoin to the
    * right, to reflect the filter now being applied on the resulting
-   * MultiJoinRel.
+   * MultiJoin.
    *
-   * @param joinRel     the original JoinRel
-   * @param left        the left child of the JoinRel
-   * @param right       the right child of the JoinRel
+   * @param joinRel     the original LogicalJoin
+   * @param left        the left child of the LogicalJoin
+   * @param right       the right child of the LogicalJoin
    * @param rightFilter the filter originating from the right child
    * @return the adjusted right filter
    */
   private RexNode shiftRightFilter(
-      JoinRelBase joinRel,
+      Join joinRel,
       RelNode left,
-      MultiJoinRel right,
+      MultiJoin right,
       RexNode rightFilter) {
     if (rightFilter == null) {
       return null;
@@ -434,8 +453,8 @@ public class ConvertMultiJoinRule extends RelOptRule {
    * Adds on to the existing join condition reference counts the references
    * from the new join condition.
    *
-   * @param multiJoinInputs          inputs into the new MultiJoinRel
-   * @param nTotalFields             total number of fields in the MultiJoinRel
+   * @param multiJoinInputs          inputs into the new MultiJoin
+   * @param nTotalFields             total number of fields in the MultiJoin
    * @param joinCondition            the new join condition
    * @param origJoinFieldRefCounts   existing join condition reference counts
    *
@@ -461,7 +480,7 @@ public class ConvertMultiJoinRule extends RelOptRule {
       currInput++;
     }
 
-    // add on to the counts for each input into the MultiJoinRel the
+    // add on to the counts for each input into the MultiJoin the
     // reference counts computed for the current join condition
     currInput = -1;
     int startField = 0;
@@ -493,25 +512,25 @@ public class ConvertMultiJoinRule extends RelOptRule {
    * Combines the post-join filters from the left and right inputs (if they
    * are MultiJoinRels) into a single AND'd filter.
    *
-   * @param joinRel the original JoinRel
-   * @param left    left child of the JoinRel
-   * @param right   right child of the JoinRel
+   * @param joinRel the original LogicalJoin
+   * @param left    left child of the LogicalJoin
+   * @param right   right child of the LogicalJoin
    * @return combined post-join filters AND'd together
    */
   private List<RexNode> combinePostJoinFilters(
-      JoinRelBase joinRel,
+      Join joinRel,
       RelNode left,
       RelNode right) {
     final List<RexNode> filters = Lists.newArrayList();
-    if (right instanceof MultiJoinRel) {
-      final MultiJoinRel multiRight = (MultiJoinRel) right;
+    if (right instanceof MultiJoin) {
+      final MultiJoin multiRight = (MultiJoin) right;
       filters.add(
           shiftRightFilter(joinRel, left, multiRight,
               multiRight.getPostJoinFilter()));
     }
 
-    if (left instanceof MultiJoinRel) {
-      filters.add(((MultiJoinRel) left).getPostJoinFilter());
+    if (left instanceof MultiJoin) {
+      filters.add(((MultiJoin) left).getPostJoinFilter());
     }
 
     return filters;
@@ -537,4 +556,4 @@ public class ConvertMultiJoinRule extends RelOptRule {
   }
 }
 
-// End ConvertMultiJoinRule.java
+// End JoinToMultiJoinRule.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/rules/JoinUnionTransposeRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/JoinUnionTransposeRule.java b/core/src/main/java/org/apache/calcite/rel/rules/JoinUnionTransposeRule.java
index 063f96b..c0fa8af 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/JoinUnionTransposeRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/JoinUnionTransposeRule.java
@@ -14,42 +14,50 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
+package org.apache.calcite.rel.rules;
 
-import java.util.*;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptRuleOperand;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.SetOp;
+import org.apache.calcite.rel.core.Union;
+import org.apache.calcite.rel.logical.LogicalUnion;
 
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
+import java.util.ArrayList;
+import java.util.List;
 
 /**
- * PushJoinThroughUnionRule implements the rule for pushing a
- * {@link JoinRel} past a non-distinct {@link UnionRel}.
+ * Planner rule that pushes a
+ * {@link org.apache.calcite.rel.logical.LogicalJoin}
+ * past a non-distinct {@link org.apache.calcite.rel.logical.LogicalUnion}.
  */
-public class PushJoinThroughUnionRule extends RelOptRule {
-  public static final PushJoinThroughUnionRule LEFT_UNION =
-      new PushJoinThroughUnionRule(
-          operand(JoinRelBase.class,
-              operand(UnionRelBase.class, any()),
+public class JoinUnionTransposeRule extends RelOptRule {
+  public static final JoinUnionTransposeRule LEFT_UNION =
+      new JoinUnionTransposeRule(
+          operand(Join.class,
+              operand(Union.class, any()),
               operand(RelNode.class, any())),
           "union on left");
 
-  public static final PushJoinThroughUnionRule RIGHT_UNION =
-      new PushJoinThroughUnionRule(
-          operand(JoinRelBase.class,
+  public static final JoinUnionTransposeRule RIGHT_UNION =
+      new JoinUnionTransposeRule(
+          operand(Join.class,
               operand(RelNode.class, any()),
-              operand(UnionRelBase.class, any())),
+              operand(Union.class, any())),
           "union on right");
 
-  private PushJoinThroughUnionRule(RelOptRuleOperand operand, String id) {
-    super(operand, "PushJoinThroughUnionRule: " + id);
+  private JoinUnionTransposeRule(RelOptRuleOperand operand, String id) {
+    super(operand, "JoinUnionTransposeRule: " + id);
   }
 
   public void onMatch(RelOptRuleCall call) {
-    final JoinRelBase join = call.rel(0);
-    final UnionRelBase unionRel;
+    final Join join = call.rel(0);
+    final Union unionRel;
     RelNode otherInput;
     boolean unionOnLeft;
-    if (call.rel(1) instanceof UnionRel) {
+    if (call.rel(1) instanceof LogicalUnion) {
       unionRel = call.rel(1);
       otherInput = call.rel(2);
       unionOnLeft = true;
@@ -97,10 +105,10 @@ public class PushJoinThroughUnionRule extends RelOptRule {
               join.getJoinType(),
               join.isSemiJoinDone()));
     }
-    final SetOpRel newUnionRel =
+    final SetOp newUnionRel =
         unionRel.copy(unionRel.getTraitSet(), newUnionInputs, true);
     call.transformTo(newUnionRel);
   }
 }
 
-// End PushJoinThroughUnionRule.java
+// End JoinUnionTransposeRule.java


[17/58] [abbrv] [partial] incubator-calcite git commit: [CALCITE-306] Standardize code style for "import package.*; "

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/sql/test/SqlAdvisorTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/test/SqlAdvisorTest.java b/core/src/test/java/org/apache/calcite/sql/test/SqlAdvisorTest.java
index d8ebe27..ad86773 100644
--- a/core/src/test/java/org/apache/calcite/sql/test/SqlAdvisorTest.java
+++ b/core/src/test/java/org/apache/calcite/sql/test/SqlAdvisorTest.java
@@ -14,23 +14,36 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.test;
-
-import java.util.*;
-
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.advise.*;
-import org.eigenbase.sql.fun.*;
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.sql.validate.*;
-import org.eigenbase.test.*;
-import org.eigenbase.util.*;
+package org.apache.calcite.sql.test;
+
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.sql.advise.SqlAdvisor;
+import org.apache.calcite.sql.advise.SqlAdvisorValidator;
+import org.apache.calcite.sql.advise.SqlSimpleParser;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.parser.SqlParserUtil;
+import org.apache.calcite.sql.type.SqlTypeFactoryImpl;
+import org.apache.calcite.sql.validate.SqlConformance;
+import org.apache.calcite.sql.validate.SqlMoniker;
+import org.apache.calcite.sql.validate.SqlMonikerType;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql.validate.SqlValidatorWithHints;
+import org.apache.calcite.test.MockCatalogReader;
+import org.apache.calcite.test.SqlValidatorTestCase;
 
 import org.junit.Assert;
 import org.junit.Test;
 
-import static org.junit.Assert.*;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.fail;
 
 /**
  * Concrete child class of {@link SqlValidatorTestCase}, containing unit tests
@@ -285,7 +298,7 @@ public class SqlAdvisorTest extends SqlValidatorTestCase {
 
   private static final String EMPNO_EMP =
       "COLUMN(EMPNO)\n"
-      + "TABLE(EMP)\n";
+          + "TABLE(EMP)\n";
 
   //~ Constructors -----------------------------------------------------------
 
@@ -434,8 +447,8 @@ public class SqlAdvisorTest extends SqlValidatorTestCase {
         && (expectedList.size() == uniqueResults.values().size()))) {
       fail(
           "SqlAdvisorTest: completion hints results not as salesTables:\n"
-          + uniqueResults.values() + "\nExpected:\n"
-          + expectedList);
+              + uniqueResults.values() + "\nExpected:\n"
+              + expectedList);
     }
   }
 
@@ -464,8 +477,7 @@ public class SqlAdvisorTest extends SqlValidatorTestCase {
     return buf.toString();
   }
 
-  @Override
-  public SqlTester getTester() {
+  @Override public SqlTester getTester() {
     return new SqlTesterImpl(new AdvisorTesterFactory());
   }
 
@@ -518,7 +530,7 @@ public class SqlAdvisorTest extends SqlValidatorTestCase {
     // from
     sql =
         "select a.empno, b.deptno from ^dummy a join sales.dummy b "
-        + "on a.deptno=b.deptno where empno=1";
+            + "on a.deptno=b.deptno where empno=1";
     assertHint(sql, getFromKeywords(), SCHEMAS, getSalesTables());
 
     // from
@@ -530,7 +542,7 @@ public class SqlAdvisorTest extends SqlValidatorTestCase {
     // join
     sql =
         "select a.empno, b.deptno from dummy a join ^sales.dummy b "
-        + "on a.deptno=b.deptno where empno=1";
+            + "on a.deptno=b.deptno where empno=1";
     assertHint(sql, getFromKeywords(), SCHEMAS, getSalesTables());
 
     sql = "select a.empno, b.deptno from dummy a join sales.^";
@@ -556,27 +568,27 @@ public class SqlAdvisorTest extends SqlValidatorTestCase {
 
     sql =
         "select a.empno, b.deptno from sales.emp a join sales.dept b "
-        + "on ^a.deptno=b.dummy where empno=1";
+            + "on ^a.deptno=b.dummy where empno=1";
     assertHint(sql, AB_TABLES, EXPR_KEYWORDS); // on left
 
     sql =
         "select a.empno, b.deptno from sales.emp a join sales.dept b "
-        + "on a.^";
+            + "on a.^";
     assertComplete(sql, EMP_COLUMNS); // on left
 
     sql =
         "select a.empno, b.deptno from sales.emp a join sales.dept b "
-        + "on a.deptno=^b.dummy where empno=1";
+            + "on a.deptno=^b.dummy where empno=1";
     assertHint(sql, EXPR_KEYWORDS, AB_TABLES); // on right
 
     sql =
         "select a.empno, b.deptno from sales.emp a join sales.dept b "
-        + "on a.deptno=b.^ where empno=1";
+            + "on a.deptno=b.^ where empno=1";
     assertComplete(sql, DEPT_COLUMNS); // on right
 
     sql =
         "select a.empno, b.deptno from sales.emp a join sales.dept b "
-        + "on a.deptno=b.^";
+            + "on a.deptno=b.^";
     assertComplete(sql, DEPT_COLUMNS); // on right
   }
 
@@ -585,31 +597,31 @@ public class SqlAdvisorTest extends SqlValidatorTestCase {
 
     sql =
         "select a.empno, b.deptno from sales.emp a, sales.dept b "
-        + "where b.deptno=^a.dummy";
+            + "where b.deptno=^a.dummy";
     assertHint(sql, AB_TABLES, EXPR_KEYWORDS); // where list
 
     sql =
         "select a.empno, b.deptno from sales.emp a, sales.dept b "
-        + "where b.deptno=a.^";
+            + "where b.deptno=a.^";
     assertComplete(sql, EMP_COLUMNS); // where list
 
     // hints contain no columns, only table aliases, because there are >1
     // aliases
     sql =
         "select a.empno, b.deptno from sales.emp a, sales.dept b "
-        + "where ^dummy=1";
+            + "where ^dummy=1";
     assertHint(sql, AB_TABLES, EXPR_KEYWORDS); // where list
 
     sql =
         "select a.empno, b.deptno from sales.emp a, sales.dept b "
-        + "where ^";
+            + "where ^";
     assertComplete(sql, AB_TABLES, EXPR_KEYWORDS); // where list
 
     // If there's only one table alias, we allow both the alias and the
     // unqualified columns
     assertComplete(
         "select a.empno, a.deptno from sales.emp a "
-        + "where ^",
+            + "where ^",
         A_TABLE,
         EMP_COLUMNS,
         EXPR_KEYWORDS);
@@ -620,27 +632,27 @@ public class SqlAdvisorTest extends SqlValidatorTestCase {
 
     sql =
         "select a.empno, b.deptno from sales.emp a join sales.dept b "
-        + "on a.deptno=b.deptno where ^dummy=1";
+            + "on a.deptno=b.deptno where ^dummy=1";
     assertHint(sql, EXPR_KEYWORDS, AB_TABLES); // where list
 
     sql =
         "select a.empno, b.deptno from sales.emp a join sales.dept b "
-        + "on a.deptno=b.deptno where ^";
+            + "on a.deptno=b.deptno where ^";
     assertComplete(sql, EXPR_KEYWORDS, AB_TABLES); // where list
 
     sql =
         "select a.empno, b.deptno from sales.emp a join sales.dept b "
-        + "on a.deptno=b.deptno where ^a.dummy=1";
+            + "on a.deptno=b.deptno where ^a.dummy=1";
     assertHint(sql, EXPR_KEYWORDS, AB_TABLES); // where list
 
     sql =
         "select a.empno, b.deptno from sales.emp a join sales.dept b "
-        + "on a.deptno=b.deptno where a.^";
+            + "on a.deptno=b.deptno where a.^";
     assertComplete(sql, EMP_COLUMNS);
 
     sql =
         "select a.empno, b.deptno from sales.emp a join sales.dept b "
-        + "on a.deptno=b.deptno where a.empno ^ ";
+            + "on a.deptno=b.deptno where a.empno ^ ";
     assertComplete(sql, PREDICATE_KEYWORDS, WHERE_KEYWORDS);
   }
 
@@ -649,7 +661,7 @@ public class SqlAdvisorTest extends SqlValidatorTestCase {
 
     sql =
         "select ^dummy, b.dummy from sales.emp a join sales.dept b "
-        + "on a.deptno=b.deptno where empno=1";
+            + "on a.deptno=b.deptno where empno=1";
     assertHint(
         sql, getSelectKeywords(), EXPR_KEYWORDS, AB_TABLES, SETOPS,
         FETCH_OFFSET);
@@ -679,7 +691,7 @@ public class SqlAdvisorTest extends SqlValidatorTestCase {
 
     sql =
         "select dummy, ^b.dummy from sales.emp a join sales.dept b "
-        + "on a.deptno=b.deptno where empno=1";
+            + "on a.deptno=b.deptno where empno=1";
     assertHint(sql, EXPR_KEYWORDS, STAR_KEYWORD, AB_TABLES);
 
     sql = "select dummy, b.^ from sales.emp a join sales.dept b on true";
@@ -716,9 +728,9 @@ public class SqlAdvisorTest extends SqlValidatorTestCase {
 
     sql =
         "select emp.empno\n"
-        + "from sales.emp as e(\n"
-        + "  mpno,name,ob,gr,iredate,al,omm,eptno,lacker)\n"
-        + "where e.mpno=1 order by ^";
+            + "from sales.emp as e(\n"
+            + "  mpno,name,ob,gr,iredate,al,omm,eptno,lacker)\n"
+            + "where e.mpno=1 order by ^";
     assertComplete(
         sql,
         EXPR_KEYWORDS,
@@ -801,10 +813,10 @@ public class SqlAdvisorTest extends SqlValidatorTestCase {
     // with A).
     sql =
         "select * from sales.emp a where deptno in ("
-        + "select * from sales.dept b where ^)";
+            + "select * from sales.dept b where ^)";
     String simplifiedSql =
         "SELECT * FROM sales.emp a WHERE deptno in ("
-        + " SELECT * FROM sales.dept b WHERE _suggest_ )";
+            + " SELECT * FROM sales.dept b WHERE _suggest_ )";
     assertSimplify(sql, simplifiedSql);
     assertComplete(
         sql,
@@ -816,68 +828,68 @@ public class SqlAdvisorTest extends SqlValidatorTestCase {
   @Test public void testSimpleParserTokenizer() {
     String sql =
         "select"
-        + " 12"
-        + " "
-        + "*"
-        + " 1.23e45"
-        + " "
-        + "("
-        + "\"an id\""
-        + ","
-        + " "
-        + "\"an id with \"\"quotes' inside\""
-        + ","
-        + " "
-        + "/* a comment, with 'quotes', over\nmultiple lines\nand select keyword */"
-        + "\n "
-        + "("
-        + " "
-        + "a"
-        + " "
-        + "different"
-        + " "
-        + "// comment\n\r"
-        + "//and a comment /* containing comment */ and then some more\r"
-        + ")"
-        + " "
-        + "from"
-        + " "
-        + "t"
-        + ")"
-        + ")"
-        + "/* a comment after close paren */"
-        + " "
-        + "("
-        + "'quoted'"
-        + " "
-        + "'string with ''single and \"double\"\" quote'"
-        + ")";
+            + " 12"
+            + " "
+            + "*"
+            + " 1.23e45"
+            + " "
+            + "("
+            + "\"an id\""
+            + ","
+            + " "
+            + "\"an id with \"\"quotes' inside\""
+            + ","
+            + " "
+            + "/* a comment, with 'quotes', over\nmultiple lines\nand select keyword */"
+            + "\n "
+            + "("
+            + " "
+            + "a"
+            + " "
+            + "different"
+            + " "
+            + "// comment\n\r"
+            + "//and a comment /* containing comment */ and then some more\r"
+            + ")"
+            + " "
+            + "from"
+            + " "
+            + "t"
+            + ")"
+            + ")"
+            + "/* a comment after close paren */"
+            + " "
+            + "("
+            + "'quoted'"
+            + " "
+            + "'string with ''single and \"double\"\" quote'"
+            + ")";
     String expected =
         "SELECT\n"
-        + "ID(12)\n"
-        + "ID(*)\n"
-        + "ID(1.23e45)\n"
-        + "LPAREN\n"
-        + "DQID(\"an id\")\n"
-        + "COMMA\n"
-        + "DQID(\"an id with \"\"quotes' inside\")\n"
-        + "COMMA\n"
-        + "COMMENT\n"
-        + "LPAREN\n"
-        + "ID(a)\n"
-        + "ID(different)\n"
-        + "COMMENT\n"
-        + "COMMENT\n"
-        + "RPAREN\n"
-        + "FROM\n"
-        + "ID(t)\n"
-        + "RPAREN\n"
-        + "RPAREN\n"
-        + "COMMENT\n"
-        + "LPAREN\n"
-        + "SQID('quoted')\n"
-        + "SQID('string with ''single and \"double\"\" quote')\n"
-        + "RPAREN\n";
+            + "ID(12)\n"
+            + "ID(*)\n"
+            + "ID(1.23e45)\n"
+            + "LPAREN\n"
+            + "DQID(\"an id\")\n"
+            + "COMMA\n"
+            + "DQID(\"an id with \"\"quotes' inside\")\n"
+            + "COMMA\n"
+            + "COMMENT\n"
+            + "LPAREN\n"
+            + "ID(a)\n"
+            + "ID(different)\n"
+            + "COMMENT\n"
+            + "COMMENT\n"
+            + "RPAREN\n"
+            + "FROM\n"
+            + "ID(t)\n"
+            + "RPAREN\n"
+            + "RPAREN\n"
+            + "COMMENT\n"
+            + "LPAREN\n"
+            + "SQID('quoted')\n"
+            + "SQID('string with ''single and \"double\"\" quote')\n"
+            + "RPAREN\n";
     assertTokenizesTo(sql, expected);
 
     // Tokenizer should be lenient if input ends mid-token
@@ -947,16 +959,16 @@ public class SqlAdvisorTest extends SqlValidatorTestCase {
     // on
     sql =
         "select a.empno, b.deptno from sales.emp a join sales.dept b "
-        + "on a.deptno=^";
+            + "on a.deptno=^";
     expected =
         "SELECT * FROM sales.emp a JOIN sales.dept b "
-        + "ON a.deptno= _suggest_";
+            + "ON a.deptno= _suggest_";
     assertSimplify(sql, expected);
 
     // where
     sql =
         "select a.empno, b.deptno from sales.emp a, sales.dept b "
-        + "where ^";
+            + "where ^";
     expected = "SELECT * FROM sales.emp a , sales.dept b WHERE _suggest_";
     assertSimplify(sql, expected);
 
@@ -968,27 +980,27 @@ public class SqlAdvisorTest extends SqlValidatorTestCase {
     // subquery in from
     sql =
         "select t.^ from (select 1 as x, 2 as y from sales.emp) as t "
-        + "where t.dummy=1";
+            + "where t.dummy=1";
     expected =
         "SELECT t. _suggest_ "
-        + "FROM ( SELECT 0 AS x , 0 AS y FROM sales.emp ) as t";
+            + "FROM ( SELECT 0 AS x , 0 AS y FROM sales.emp ) as t";
     assertSimplify(sql, expected);
 
     sql =
         "select t. from (select 1 as x, 2 as y from "
-        + "(select x from sales.emp)) as t where ^";
+            + "(select x from sales.emp)) as t where ^";
     expected =
         "SELECT * FROM ( SELECT 0 AS x , 0 AS y FROM "
-        + "( SELECT 0 AS x FROM sales.emp ) ) as t WHERE _suggest_";
+            + "( SELECT 0 AS x FROM sales.emp ) ) as t WHERE _suggest_";
     assertSimplify(sql, expected);
 
     sql =
         "select ^from (select 1 as x, 2 as y from sales.emp), "
-        + "(select 2 as y from (select m from n where)) as t "
-        + "where t.dummy=1";
+            + "(select 2 as y from (select m from n where)) as t "
+            + "where t.dummy=1";
     expected =
         "SELECT _suggest_ FROM ( SELECT 0 AS x , 0 AS y FROM sales.emp ) "
-        + ", ( SELECT 0 AS y FROM ( SELECT 0 AS m FROM n ) ) as t";
+            + ", ( SELECT 0 AS y FROM ( SELECT 0 AS m FROM n ) ) as t";
     assertSimplify(sql, expected);
 
     // Note: completes the missing close paren; wipes out select clause of
@@ -1009,11 +1021,11 @@ public class SqlAdvisorTest extends SqlValidatorTestCase {
     // 4. removes SELECT clause of outer query.
     sql =
         "select x + y + 32 from "
-        + "(select 1 as x, 2 as y from sales group by invalid stuff) as t "
-        + "where x in (select deptno from emp where foo + t.^ < 10)";
+            + "(select 1 as x, 2 as y from sales group by invalid stuff) as t "
+            + "where x in (select deptno from emp where foo + t.^ < 10)";
     expected =
         "SELECT * FROM ( SELECT 0 AS x , 0 AS y FROM sales ) as t "
-        + "WHERE x in ( SELECT * FROM emp WHERE foo + t. _suggest_ < 10 )";
+            + "WHERE x in ( SELECT * FROM emp WHERE foo + t. _suggest_ < 10 )";
     assertSimplify(sql, expected);
 
     // if hint is in FROM, can remove other members of FROM clause
@@ -1028,26 +1040,26 @@ public class SqlAdvisorTest extends SqlValidatorTestCase {
 
     sql =
         "select count(1) from sales.emp a "
-        + "where substring(a.^ FROM 3 for 6) = '1234'";
+            + "where substring(a.^ FROM 3 for 6) = '1234'";
     expected =
         "SELECT * FROM sales.emp a "
-        + "WHERE substring ( a. _suggest_ FROM 3 for 6 ) = '1234'";
+            + "WHERE substring ( a. _suggest_ FROM 3 for 6 ) = '1234'";
     assertSimplify(sql, expected);
 
     // missing ')' following subquery
     sql =
         "select * from sales.emp a where deptno in ("
-        + "select * from sales.dept b where ^";
+            + "select * from sales.dept b where ^";
     expected =
         "SELECT * FROM sales.emp a WHERE deptno in ("
-        + " SELECT * FROM sales.dept b WHERE _suggest_ )";
+            + " SELECT * FROM sales.dept b WHERE _suggest_ )";
     assertSimplify(sql, expected);
 
     // keyword embedded in single and double quoted string should be
     // ignored
     sql =
         "select 'a cat from a king' as foobar, 1 / 2 \"where\" from t "
-        + "group by t.^ order by 123";
+            + "group by t.^ order by 123";
     expected = "SELECT * FROM t GROUP BY t. _suggest_";
     assertSimplify(sql, expected);
 
@@ -1088,12 +1100,12 @@ public class SqlAdvisorTest extends SqlValidatorTestCase {
     String sql = "select * from emp where e^ and emp.deptno = 10";
     final String expected =
         "COLUMN(EMPNO)\n"
-        + "COLUMN(ENAME)\n"
-        + "KEYWORD(ELEMENT)\n"
-        + "KEYWORD(EXISTS)\n"
-        + "KEYWORD(EXP)\n"
-        + "KEYWORD(EXTRACT)\n"
-        + "TABLE(EMP)\n";
+            + "COLUMN(ENAME)\n"
+            + "KEYWORD(ELEMENT)\n"
+            + "KEYWORD(EXISTS)\n"
+            + "KEYWORD(EXP)\n"
+            + "KEYWORD(EXTRACT)\n"
+            + "TABLE(EMP)\n";
     assertComplete(sql, expected, "e");
 
     // cursor in middle of word and at end
@@ -1174,13 +1186,13 @@ public class SqlAdvisorTest extends SqlValidatorTestCase {
     assertSimplify(sql, simplified);
   }
 
+  /** Factory that creates testers. */
   private static class AdvisorTesterFactory extends DelegatingSqlTestFactory {
     public AdvisorTesterFactory() {
       super(DefaultSqlTestFactory.INSTANCE);
     }
 
-    @Override
-    public SqlValidator getValidator(SqlTestFactory factory) {
+    @Override public SqlValidator getValidator(SqlTestFactory factory) {
       final RelDataTypeFactory typeFactory =
           new SqlTypeFactoryImpl(RelDataTypeSystem.DEFAULT);
       final SqlConformance conformance = (SqlConformance) get("conformance");
@@ -1192,8 +1204,7 @@ public class SqlAdvisorTest extends SqlValidatorTestCase {
           conformance);
     }
 
-    @Override
-    public SqlAdvisor createAdvisor(SqlValidatorWithHints validator) {
+    @Override public SqlAdvisor createAdvisor(SqlValidatorWithHints validator) {
       return new SqlAdvisor(validator);
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/sql/test/SqlOperatorBaseTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/test/SqlOperatorBaseTest.java b/core/src/test/java/org/apache/calcite/sql/test/SqlOperatorBaseTest.java
index c1ce991..8987b75 100644
--- a/core/src/test/java/org/apache/calcite/sql/test/SqlOperatorBaseTest.java
+++ b/core/src/test/java/org/apache/calcite/sql/test/SqlOperatorBaseTest.java
@@ -14,33 +14,45 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.test;
-
-import java.math.*;
-
-import java.sql.*;
-import java.text.*;
-
-import java.util.*;
-import java.util.regex.*;
-
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.fun.*;
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.sql.util.SqlString;
-import org.eigenbase.test.*;
-import org.eigenbase.util.*;
-
-import net.hydromatic.optiq.runtime.Hook;
-import net.hydromatic.optiq.test.OptiqAssert;
+package org.apache.calcite.sql.test;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.runtime.Hook;
+import org.apache.calcite.sql.SqlDialect;
+import org.apache.calcite.sql.SqlJdbcFunctionCall;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.BasicSqlType;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.util.SqlString;
+import org.apache.calcite.test.CalciteAssert;
+import org.apache.calcite.test.SqlLimitsTest;
+import org.apache.calcite.util.Bug;
+import org.apache.calcite.util.Holder;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
 
 import com.google.common.base.Function;
 
-import org.junit.*;
-
-import static org.junit.Assert.*;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.math.BigDecimal;
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Calendar;
+import java.util.List;
+import java.util.TimeZone;
+import java.util.regex.Pattern;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
 
 /**
  * Contains unit tests for all operators. Each of the methods is named after an
@@ -63,7 +75,7 @@ import static org.junit.Assert.*;
  *
  * <p>A typical method will be named after the operator it is testing (say
  * <code>testSubstringFunc</code>). It first calls
- * {@link SqlTester#setFor(org.eigenbase.sql.SqlOperator, org.eigenbase.sql.test.SqlTester.VmName...)}
+ * {@link SqlTester#setFor(org.apache.calcite.sql.SqlOperator, org.apache.calcite.sql.test.SqlTester.VmName...)}
  * to declare which operator it is testing.
  *
  * <blockquote>
@@ -142,7 +154,7 @@ public abstract class SqlOperatorBaseTest {
   public static final Pattern TIMESTAMP_PATTERN =
       Pattern.compile(
           "[0-9][0-9][0-9][0-9]-[0-9][0-9]-[0-9][0-9] "
-          + "[0-9][0-9]:[0-9][0-9]:[0-9][0-9]");
+              + "[0-9][0-9]:[0-9][0-9]:[0-9][0-9]");
 
   /**
    * Regular expression for a SQL DATE value.
@@ -1315,7 +1327,7 @@ public abstract class SqlOperatorBaseTest {
         "CHAR(3)");
     tester.checkString(
         "case 1 when 1 then cast('a' as varchar(1)) "
-        + "when 2 then cast('bcd' as varchar(3)) end",
+            + "when 2 then cast('bcd' as varchar(3)) end",
         "a",
         "VARCHAR(3)");
     if (DECIMAL) {
@@ -1389,29 +1401,29 @@ public abstract class SqlOperatorBaseTest {
     // multiple values in some cases (introduced in SQL:2011)
     tester.checkString(
         "case 1 "
-        + "when 1, 2 then '1 or 2' "
-        + "when 2 then 'not possible' "
-        + "when 3, 2 then '3' "
-        + "else 'none of the above' "
-        + "end",
+            + "when 1, 2 then '1 or 2' "
+            + "when 2 then 'not possible' "
+            + "when 3, 2 then '3' "
+            + "else 'none of the above' "
+            + "end",
         "1 or 2           ",
         "CHAR(17) NOT NULL");
     tester.checkString(
         "case 2 "
-        + "when 1, 2 then '1 or 2' "
-        + "when 2 then 'not possible' "
-        + "when 3, 2 then '3' "
-        + "else 'none of the above' "
-        + "end",
+            + "when 1, 2 then '1 or 2' "
+            + "when 2 then 'not possible' "
+            + "when 3, 2 then '3' "
+            + "else 'none of the above' "
+            + "end",
         "1 or 2           ",
         "CHAR(17) NOT NULL");
     tester.checkString(
         "case 3 "
-        + "when 1, 2 then '1 or 2' "
-        + "when 2 then 'not possible' "
-        + "when 3, 2 then '3' "
-        + "else 'none of the above' "
-        + "end",
+            + "when 1, 2 then '1 or 2' "
+            + "when 2 then 'not possible' "
+            + "when 3, 2 then '3' "
+            + "else 'none of the above' "
+            + "end",
         "3                ",
         "CHAR(17) NOT NULL");
     tester.checkString(
@@ -1714,26 +1726,26 @@ public abstract class SqlOperatorBaseTest {
           "RecordType(INTEGER NOT NULL EXPR$0, INTEGER EXPR$1) NOT NULL");
       tester.checkType(
           "SELECT *,(SELECT * FROM (VALUES(CAST(10 as BIGINT)))) "
-          + "FROM (VALUES(CAST(10 as bigint)))",
+              + "FROM (VALUES(CAST(10 as bigint)))",
           "RecordType(BIGINT NOT NULL EXPR$0, BIGINT EXPR$1) NOT NULL");
       tester.checkType(
           " SELECT *,(SELECT * FROM (VALUES(10.5))) FROM (VALUES(10.5))",
           "RecordType(DECIMAL(3, 1) NOT NULL EXPR$0, DECIMAL(3, 1) EXPR$1) NOT NULL");
       tester.checkType(
           "SELECT *,(SELECT * FROM (VALUES('this is a char'))) "
-          + "FROM (VALUES('this is a char too'))",
+              + "FROM (VALUES('this is a char too'))",
           "RecordType(CHAR(18) NOT NULL EXPR$0, CHAR(14) EXPR$1) NOT NULL");
       tester.checkType(
           "SELECT *,(SELECT * FROM (VALUES(true))) FROM (values(false))",
           "RecordType(BOOLEAN NOT NULL EXPR$0, BOOLEAN EXPR$1) NOT NULL");
       tester.checkType(
           " SELECT *,(SELECT * FROM (VALUES(cast('abcd' as varchar(10))))) "
-          + "FROM (VALUES(CAST('abcd' as varchar(10))))",
+              + "FROM (VALUES(CAST('abcd' as varchar(10))))",
           "RecordType(VARCHAR(10) NOT NULL EXPR$0, VARCHAR(10) EXPR$1) NOT NULL");
       tester.checkType(
           "SELECT *,"
-          + "  (SELECT * FROM (VALUES(TIMESTAMP '2006-01-01 12:00:05'))) "
-          + "FROM (VALUES(TIMESTAMP '2006-01-01 12:00:05'))",
+              + "  (SELECT * FROM (VALUES(TIMESTAMP '2006-01-01 12:00:05'))) "
+              + "FROM (VALUES(TIMESTAMP '2006-01-01 12:00:05'))",
           "RecordType(TIMESTAMP(0) NOT NULL EXPR$0, TIMESTAMP(0) EXPR$1) NOT NULL");
     }
   }
@@ -2487,30 +2499,30 @@ public abstract class SqlOperatorBaseTest {
     // combine '<datetime> + <interval>' with '<datetime> - <datetime>'
     tester.checkScalar(
         "timestamp '1969-04-29 0:0:0' +"
-        + " (timestamp '2008-07-15 15:28:00' - "
-        + "  timestamp '1969-04-29 0:0:0') day to second / 2",
+            + " (timestamp '2008-07-15 15:28:00' - "
+            + "  timestamp '1969-04-29 0:0:0') day to second / 2",
         "1988-12-06 07:44:00",
         "TIMESTAMP(0) NOT NULL");
 
     tester.checkScalar(
         "date '1969-04-29' +"
-        + " (date '2008-07-15' - "
-        + "  date '1969-04-29') day / 2",
+            + " (date '2008-07-15' - "
+            + "  date '1969-04-29') day / 2",
         "1988-12-06",
         "DATE NOT NULL");
 
     tester.checkScalar(
         "time '01:23:44' +"
-        + " (time '15:28:00' - "
-        + "  time '01:23:44') hour to second / 2",
+            + " (time '15:28:00' - "
+            + "  time '01:23:44') hour to second / 2",
         "08:25:52",
         "TIME(0) NOT NULL");
 
     if (Bug.DT1684_FIXED) {
       tester.checkBoolean(
           "(date '1969-04-29' +"
-          + " (CURRENT_DATE - "
-          + "  date '1969-04-29') day / 2) is not null",
+              + " (CURRENT_DATE - "
+              + "  date '1969-04-29') day / 2) is not null",
           Boolean.TRUE);
     }
     // TODO: Add tests for year month intervals (currently not supported)
@@ -3195,15 +3207,15 @@ public abstract class SqlOperatorBaseTest {
     tester.checkFails(
         "'yd' similar to '[x-ze-a]d'",
         "Illegal character range near index 6\n"
-        + "\\[x-ze-a\\]d\n"
-        + "      \\^",
+            + "\\[x-ze-a\\]d\n"
+            + "      \\^",
         true);   // illegal range
 
     tester.checkFails(
         "'yd3223' similar to '[:LOWER:]{2}[:DIGIT:]{,5}'",
         "Illegal repetition near index 20\n"
-        + "\\[\\:LOWER\\:\\]\\{2\\}\\[\\:DIGIT\\:\\]\\{,5\\}\n"
-        + "                    \\^",
+            + "\\[\\:LOWER\\:\\]\\{2\\}\\[\\:DIGIT\\:\\]\\{,5\\}\n"
+            + "                    \\^",
         true);
 
     tester.checkFails(
@@ -3273,7 +3285,7 @@ public abstract class SqlOperatorBaseTest {
 
       tester.checkBoolean(
           "'YD  3223' similar to "
-          + "'[:UPPER:]{2}||[:WHITESPACE:]*[:DIGIT:]{4}'",
+              + "'[:UPPER:]{2}||[:WHITESPACE:]*[:DIGIT:]{4}'",
           Boolean.TRUE);
 
       tester.checkBoolean(
@@ -3282,12 +3294,12 @@ public abstract class SqlOperatorBaseTest {
 
       tester.checkBoolean(
           "'YD\t3223' similar to "
-          + "'[:UPPER:]{2}[:WHITESPACE:]*[:DIGIT:]{4}'",
+              + "'[:UPPER:]{2}[:WHITESPACE:]*[:DIGIT:]{4}'",
           Boolean.TRUE);
 
       tester.checkBoolean(
           "'YD\t\t3223' similar to "
-          + "'([:UPPER:]{2}[:WHITESPACE:]+)||[:DIGIT:]{4}'",
+              + "'([:UPPER:]{2}[:WHITESPACE:]+)||[:DIGIT:]{4}'",
           Boolean.TRUE);
     }
   }
@@ -3323,14 +3335,14 @@ public abstract class SqlOperatorBaseTest {
     if (enable) {
       tester.checkString(
           "overlay(cast('ABCdef' as varchar(10)) placing "
-          + "cast('abc' as char(5)) from 1 for 2)",
+              + "cast('abc' as char(5)) from 1 for 2)",
           "abc  Cdef",
           "VARCHAR(15) NOT NULL");
     }
     if (enable) {
       tester.checkString(
           "overlay(cast('ABCdef' as char(10)) placing "
-          + "cast('abc' as char(5)) from 1 for 2)",
+              + "cast('abc' as char(5)) from 1 for 2)",
           "abc  Cdef    ",
           "VARCHAR(15) NOT NULL");
     }
@@ -3350,14 +3362,14 @@ public abstract class SqlOperatorBaseTest {
     if (enable) {
       tester.checkString(
           "overlay(cast(x'ABCdef' as varbinary(5)) placing "
-          + "cast(x'abcd' as binary(3)) from 1 for 2)",
+              + "cast(x'abcd' as binary(3)) from 1 for 2)",
           "abc  Cdef",
           "VARBINARY(8) NOT NULL");
     }
     if (enable) {
       tester.checkString(
           "overlay(cast(x'ABCdef' as binary(5)) placing "
-          + "cast(x'abcd' as binary(3)) from 1 for 2)",
+              + "cast(x'abcd' as binary(3)) from 1 for 2)",
           "abc  Cdef    ",
           "VARBINARY(8) NOT NULL");
     }
@@ -3872,7 +3884,7 @@ public abstract class SqlOperatorBaseTest {
   protected static Pair<String, Hook.Closeable> currentTimeString(TimeZone tz) {
     final Calendar calendar;
     final Hook.Closeable closeable;
-    if (OptiqAssert.ENABLE_SLOW) {
+    if (CalciteAssert.ENABLE_SLOW) {
       calendar = getCalendarNotTooNear(Calendar.HOUR_OF_DAY);
       closeable = new Hook.Closeable() {
         public void close() {}
@@ -3980,16 +3992,16 @@ public abstract class SqlOperatorBaseTest {
       tester.checkFails(
           "trim('xy' from 'abcde')",
           "could not calculate results for the following row:\n"
-          + "\\[ 0 \\]\n"
-          + "Messages:\n"
-          + "\\[0\\]:PC=0 Code=22027 ",
+              + "\\[ 0 \\]\n"
+              + "Messages:\n"
+              + "\\[0\\]:PC=0 Code=22027 ",
           true);
       tester.checkFails(
           "trim('' from 'abcde')",
           "could not calculate results for the following row:\n"
-          + "\\[ 0 \\]\n"
-          + "Messages:\n"
-          + "\\[0\\]:PC=0 Code=22027 ",
+              + "\\[ 0 \\]\n"
+              + "Messages:\n"
+              + "\\[0\\]:PC=0 Code=22027 ",
           true);
     }
   }
@@ -4182,7 +4194,7 @@ public abstract class SqlOperatorBaseTest {
     tester.checkFails(
         "^ARRAY ['foo', 'bar']['baz']^",
         "Cannot apply 'ITEM' to arguments of type 'ITEM\\(<CHAR\\(3\\) ARRAY>, <CHAR\\(3\\)>\\)'\\. Supported form\\(s\\): <ARRAY>\\[<INTEGER>\\]\n"
-        + "<MAP>\\[<VALUE>\\]",
+            + "<MAP>\\[<VALUE>\\]",
         false);
 
     // Array of INTEGER NOT NULL is interesting because we might be tempted
@@ -4934,7 +4946,7 @@ public abstract class SqlOperatorBaseTest {
             literal.toSqlString(SqlDialect.DUMMY);
         final String expr =
             "CAST(" + literalString
-            + " AS " + type + ")";
+                + " AS " + type + ")";
         try {
           tester.checkType(
               expr,
@@ -5136,7 +5148,7 @@ public abstract class SqlOperatorBaseTest {
   }
 
   /**
-   * Creates a {@link org.eigenbase.sql.test.SqlTester} based on a JDBC
+   * Creates a {@link org.apache.calcite.sql.test.SqlTester} based on a JDBC
    * connection.
    */
   public static SqlTester tester(Connection connection) {
@@ -5144,7 +5156,7 @@ public abstract class SqlOperatorBaseTest {
   }
 
   /**
-   * Implementation of {@link org.eigenbase.sql.test.SqlTester} based on a
+   * Implementation of {@link org.apache.calcite.sql.test.SqlTester} based on a
    * JDBC connection.
    */
   protected static class TesterImpl extends SqlTesterImpl {
@@ -5155,8 +5167,7 @@ public abstract class SqlOperatorBaseTest {
       this.connection = connection;
     }
 
-    @Override
-    public void check(
+    @Override public void check(
         String query,
         TypeChecker typeChecker,
         ResultChecker resultChecker) {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/sql/test/SqlOperatorTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/test/SqlOperatorTest.java b/core/src/test/java/org/apache/calcite/sql/test/SqlOperatorTest.java
index 70c4544..91bc0f8 100644
--- a/core/src/test/java/org/apache/calcite/sql/test/SqlOperatorTest.java
+++ b/core/src/test/java/org/apache/calcite/sql/test/SqlOperatorTest.java
@@ -14,10 +14,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.test;
+package org.apache.calcite.sql.test;
 
-import org.eigenbase.sql.validate.*;
-import org.eigenbase.test.*;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.test.SqlValidatorTestCase;
 
 /**
  * Concrete subclass of {@link SqlOperatorBaseTest} which checks against

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/sql/test/SqlPrettyWriterTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/test/SqlPrettyWriterTest.java b/core/src/test/java/org/apache/calcite/sql/test/SqlPrettyWriterTest.java
index 5963ceb..26df28d 100644
--- a/core/src/test/java/org/apache/calcite/sql/test/SqlPrettyWriterTest.java
+++ b/core/src/test/java/org/apache/calcite/sql/test/SqlPrettyWriterTest.java
@@ -14,18 +14,23 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.test;
+package org.apache.calcite.sql.test;
 
-import java.io.*;
-
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.sql.pretty.*;
-import org.eigenbase.test.*;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlDialect;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.parser.SqlParseException;
+import org.apache.calcite.sql.parser.SqlParser;
+import org.apache.calcite.sql.pretty.SqlPrettyWriter;
+import org.apache.calcite.test.DiffRepository;
 
 import org.junit.Test;
 
-import static org.junit.Assert.*;
+import java.io.PrintWriter;
+import java.io.StringWriter;
+
+import static org.junit.Assert.assertTrue;
 
 /**
  * Unit test for {@link SqlPrettyWriter}.
@@ -58,8 +63,7 @@ public class SqlPrettyWriterTest {
     try {
       node = SqlParser.create(sql).parseQuery();
     } catch (SqlParseException e) {
-      String message =
-          "Received error while parsing SQL '" + sql
+      String message = "Received error while parsing SQL '" + sql
           + "'; error is:" + NL + e.toString();
       throw new AssertionError(message);
     }
@@ -117,8 +121,7 @@ public class SqlPrettyWriterTest {
       String expectedDesc,
       String expected) throws Exception {
     final SqlNode node =
-        parseQuery(
-            "select x as a, b as b, c as c, d,"
+        parseQuery("select x as a, b as b, c as c, d,"
             + " 'mixed-Case string',"
             + " unquotedCamelCaseId,"
             + " \"quoted id\" "
@@ -223,16 +226,16 @@ public class SqlPrettyWriterTest {
         true,
         "case 1 when 2 + 3 then 4 when case a when b then c else d end then 6 else 7 end",
         "CASE" + NL
-        + "WHEN 1 = 2 + 3" + NL
-        + "THEN 4" + NL
-        + "WHEN 1 = CASE" + NL
-        + "        WHEN `A` = `B`" + NL // todo: indent should be 4 not 8
-        + "        THEN `C`" + NL
-        + "        ELSE `D`" + NL
-        + "        END" + NL
-        + "THEN 6" + NL
-        + "ELSE 7" + NL
-        + "END");
+            + "WHEN 1 = 2 + 3" + NL
+            + "THEN 4" + NL
+            + "WHEN 1 = CASE" + NL
+            + "        WHEN `A` = `B`" + NL // todo: indent should be 4 not 8
+            + "        THEN `C`" + NL
+            + "        ELSE `D`" + NL
+            + "        END" + NL
+            + "THEN 6" + NL
+            + "ELSE 7" + NL
+            + "END");
   }
 
   @Test public void testCase2() {
@@ -262,7 +265,7 @@ public class SqlPrettyWriterTest {
     assertExprPrintsTo(
         true,
         "'x' /* comment */ 'y'" + NL
-        + "  'z' ",
+            + "  'z' ",
         "'x'" + NL + "'y'" + NL + "'z'");
   }
 
@@ -277,11 +280,11 @@ public class SqlPrettyWriterTest {
     assertPrintsTo(
         true,
         "select * from t "
-        + "union select * from ("
-        + "  select * from u "
-        + "  union select * from v) "
-        + "union select * from w "
-        + "order by a, b",
+            + "union select * from ("
+            + "  select * from u "
+            + "  union select * from v) "
+            + "union select * from w "
+            + "order by a, b",
 
         // todo: SELECT should not be indended from UNION, like this:
         // UNION
@@ -308,19 +311,19 @@ public class SqlPrettyWriterTest {
   @Test public void testWhereListItemsOnSeparateLinesOr() throws Exception {
     checkPrettySeparateLines(
         "select x"
-        + " from y"
-        + " where h is not null and i < j"
-        + " or ((a or b) is true) and d not in (f,g)"
-        + " or x <> z");
+            + " from y"
+            + " where h is not null and i < j"
+            + " or ((a or b) is true) and d not in (f,g)"
+            + " or x <> z");
   }
 
   @Test public void testWhereListItemsOnSeparateLinesAnd() throws Exception {
     checkPrettySeparateLines(
         "select x"
-        + " from y"
-        + " where h is not null and (i < j"
-        + " or ((a or b) is true)) and (d not in (f,g)"
-        + " or v <> ((w * x) + y) * z)");
+            + " from y"
+            + " where h is not null and (i < j"
+            + " or ((a or b) is true)) and (d not in (f,g)"
+            + " or v <> ((w * x) + y) * z)");
   }
 
   private void checkPrettySeparateLines(String sql) {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/sql/test/SqlTestFactory.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/test/SqlTestFactory.java b/core/src/test/java/org/apache/calcite/sql/test/SqlTestFactory.java
index 9c5e2a2..7aadf37 100644
--- a/core/src/test/java/org/apache/calcite/sql/test/SqlTestFactory.java
+++ b/core/src/test/java/org/apache/calcite/sql/test/SqlTestFactory.java
@@ -14,18 +14,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.test;
+package org.apache.calcite.sql.test;
 
-import org.eigenbase.sql.SqlOperatorTable;
-import org.eigenbase.sql.advise.SqlAdvisor;
-import org.eigenbase.sql.parser.SqlParser;
-import org.eigenbase.sql.validate.SqlValidator;
-import org.eigenbase.sql.validate.SqlValidatorWithHints;
+import org.apache.calcite.sql.SqlOperatorTable;
+import org.apache.calcite.sql.advise.SqlAdvisor;
+import org.apache.calcite.sql.parser.SqlParser;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql.validate.SqlValidatorWithHints;
 
 /**
 * Creates the objects needed to run a SQL parsing or validation test.
  *
- * @see org.eigenbase.sql.test.SqlTester
+ * @see org.apache.calcite.sql.test.SqlTester
 */
 public interface SqlTestFactory {
   SqlOperatorTable createOperatorTable();

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/sql/test/SqlTester.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/test/SqlTester.java b/core/src/test/java/org/apache/calcite/sql/test/SqlTester.java
index a6f27f9..a6790ab 100644
--- a/core/src/test/java/org/apache/calcite/sql/test/SqlTester.java
+++ b/core/src/test/java/org/apache/calcite/sql/test/SqlTester.java
@@ -14,21 +14,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.test;
+package org.apache.calcite.sql.test;
+
+import org.apache.calcite.avatica.Casing;
+import org.apache.calcite.avatica.Quoting;
+import org.apache.calcite.config.Lex;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.validate.SqlConformance;
+import org.apache.calcite.test.SqlValidatorTestCase;
 
 import java.io.Closeable;
 import java.sql.ResultSet;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.validate.SqlConformance;
-import org.eigenbase.test.SqlValidatorTestCase;
-
-import net.hydromatic.avatica.Casing;
-import net.hydromatic.avatica.Quoting;
-
-import net.hydromatic.optiq.config.Lex;
-
 /**
  * SqlTester defines a callback for testing SQL queries and expressions.
  *
@@ -363,10 +361,12 @@ public interface SqlTester extends Closeable, SqlValidatorTestCase.Tester {
 
   //~ Inner Interfaces -------------------------------------------------------
 
+  /** Type checker. */
   interface TypeChecker {
     void checkType(RelDataType type);
   }
 
+  /** Result checker. */
   interface ResultChecker {
     void checkResult(ResultSet result) throws Exception;
   }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/sql/test/SqlTesterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/test/SqlTesterImpl.java b/core/src/test/java/org/apache/calcite/sql/test/SqlTesterImpl.java
index c0c5a54..874655b 100644
--- a/core/src/test/java/org/apache/calcite/sql/test/SqlTesterImpl.java
+++ b/core/src/test/java/org/apache/calcite/sql/test/SqlTesterImpl.java
@@ -14,38 +14,58 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.test;
-
-import java.nio.charset.Charset;
-import java.util.*;
-
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.reltype.RelDataTypeField;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.fun.SqlStdOperatorTable;
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.sql.type.SqlTypeName;
-import org.eigenbase.sql.util.SqlShuttle;
-import org.eigenbase.sql.validate.*;
-import org.eigenbase.test.SqlValidatorTestCase;
-import org.eigenbase.util.*;
-
-import net.hydromatic.avatica.Casing;
-import net.hydromatic.avatica.Quoting;
-
-import net.hydromatic.optiq.config.Lex;
-import net.hydromatic.optiq.runtime.Utilities;
+package org.apache.calcite.sql.test;
+
+import org.apache.calcite.avatica.Casing;
+import org.apache.calcite.avatica.Quoting;
+import org.apache.calcite.config.Lex;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.runtime.Utilities;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlCollation;
+import org.apache.calcite.sql.SqlDialect;
+import org.apache.calcite.sql.SqlIntervalLiteral;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlSelect;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.parser.SqlParseException;
+import org.apache.calcite.sql.parser.SqlParser;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.parser.SqlParserUtil;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.util.SqlShuttle;
+import org.apache.calcite.sql.validate.SqlConformance;
+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.test.SqlValidatorTestCase;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.TestUtil;
+import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
 
-import static org.eigenbase.sql.SqlUtil.stripAs;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.NoSuchElementException;
+
+import static org.apache.calcite.sql.SqlUtil.stripAs;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.fail;
 
 /**
- * Implementation of {@link org.eigenbase.test.SqlValidatorTestCase.Tester}
+ * Implementation of {@link org.apache.calcite.test.SqlValidatorTestCase.Tester}
  * that talks to a mock catalog.
  */
 public class SqlTesterImpl implements SqlTester {
@@ -97,8 +117,8 @@ public class SqlTesterImpl implements SqlTester {
         e.printStackTrace();
         throw new AssertionError(
             "Error did not match expected ["
-            + expectedMsgPattern + "] while parsing query ["
-            + sap.sql + "]");
+                + expectedMsgPattern + "] while parsing query ["
+                + sap.sql + "]");
       }
       return;
     } catch (Throwable e) {
@@ -283,8 +303,7 @@ public class SqlTesterImpl implements SqlTester {
   private SqlTesterImpl with(final String name2, final Object value) {
     return new SqlTesterImpl(
         new DelegatingSqlTestFactory(factory) {
-          @Override
-          public Object get(String name) {
+          @Override public Object get(String name) {
             if (name.equals(name2)) {
               return value;
             }
@@ -521,8 +540,7 @@ public class SqlTesterImpl implements SqlTester {
                   SqlStdOperatorTable.CURRENT_TIME,
                   SqlStdOperatorTable.CURRENT_TIMESTAMP);
 
-          @Override
-          public SqlNode visit(SqlLiteral literal) {
+          @Override public SqlNode visit(SqlLiteral literal) {
             if (!isNull(literal)
                 && literal.getTypeName() != SqlTypeName.SYMBOL) {
               literalSet.add(literal);
@@ -530,8 +548,7 @@ public class SqlTesterImpl implements SqlTester {
             return literal;
           }
 
-          @Override
-          public SqlNode visit(SqlCall call) {
+          @Override public SqlNode visit(SqlCall call) {
             final SqlOperator operator = call.getOperator();
             if (operator == SqlStdOperatorTable.CAST
                 && isNull(call.operand(0))) {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/sql/test/SqlTests.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/test/SqlTests.java b/core/src/test/java/org/apache/calcite/sql/test/SqlTests.java
index 47a371a..9fb82b3 100644
--- a/core/src/test/java/org/apache/calcite/sql/test/SqlTests.java
+++ b/core/src/test/java/org/apache/calcite/sql/test/SqlTests.java
@@ -14,22 +14,29 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.test;
+package org.apache.calcite.sql.test;
+
+import org.apache.calcite.avatica.ColumnMetaData;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.type.SqlTypeName;
 
 import java.sql.ResultSet;
 import java.sql.Types;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
 import java.util.regex.Pattern;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.type.*;
-
-import net.hydromatic.avatica.ColumnMetaData;
-
-import static org.eigenbase.sql.test.SqlTester.*;
+import static org.apache.calcite.sql.test.SqlTester.ResultChecker;
+import static org.apache.calcite.sql.test.SqlTester.TypeChecker;
 
-import static org.hamcrest.CoreMatchers.*;
-import static org.junit.Assert.*;
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 /**
  * Utility methods.
@@ -213,13 +220,11 @@ public abstract class SqlTests {
       fail("Query returned 2 or more rows, expected 1");
     }
     if (!pattern.matcher(actual).matches()) {
-      fail(
-          "Query returned '"
+      fail("Query returned '"
               + actual
               + "', expected '"
               + pattern.pattern()
-              + "'"
-      );
+              + "'");
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/sql/test/SqlTypeNameTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/test/SqlTypeNameTest.java b/core/src/test/java/org/apache/calcite/sql/test/SqlTypeNameTest.java
index cce7df7..b8f6243 100644
--- a/core/src/test/java/org/apache/calcite/sql/test/SqlTypeNameTest.java
+++ b/core/src/test/java/org/apache/calcite/sql/test/SqlTypeNameTest.java
@@ -14,15 +14,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.test;
+package org.apache.calcite.sql.test;
 
-import java.sql.Types;
-
-import org.eigenbase.sql.type.*;
+import org.apache.calcite.sql.type.ExtraSqlTypes;
+import org.apache.calcite.sql.type.SqlTypeName;
 
 import org.junit.Test;
 
-import static org.junit.Assert.*;
+import java.sql.Types;
+
+import static org.junit.Assert.assertEquals;
 
 /**
  * Tests types supported by {@link SqlTypeName}.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/sql/test/package-info.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/test/package-info.java b/core/src/test/java/org/apache/calcite/sql/test/package-info.java
index 9a77bf1..9d3e4ec 100644
--- a/core/src/test/java/org/apache/calcite/sql/test/package-info.java
+++ b/core/src/test/java/org/apache/calcite/sql/test/package-info.java
@@ -18,6 +18,6 @@
 /**
  * Regression tests for the SQL model.
  */
-package org.eigenbase.sql.test;
+package org.apache.calcite.sql.test;
 
 // End package-info.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/test/CalciteAssert.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/CalciteAssert.java b/core/src/test/java/org/apache/calcite/test/CalciteAssert.java
index 54a306d..82931d0 100644
--- a/core/src/test/java/org/apache/calcite/test/CalciteAssert.java
+++ b/core/src/test/java/org/apache/calcite/test/CalciteAssert.java
@@ -14,50 +14,76 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.test;
-
-import net.hydromatic.optiq.*;
-import net.hydromatic.optiq.config.OptiqConnectionProperty;
-import net.hydromatic.optiq.impl.AbstractSchema;
-import net.hydromatic.optiq.impl.ViewTable;
-import net.hydromatic.optiq.impl.clone.CloneSchema;
-import net.hydromatic.optiq.impl.java.ReflectiveSchema;
-import net.hydromatic.optiq.impl.jdbc.JdbcSchema;
-import net.hydromatic.optiq.jdbc.MetaImpl;
-import net.hydromatic.optiq.jdbc.OptiqConnection;
-import net.hydromatic.optiq.jdbc.OptiqSchema;
-import net.hydromatic.optiq.materialize.Lattice;
-import net.hydromatic.optiq.runtime.Hook;
-
-import org.eigenbase.rel.RelNode;
-import org.eigenbase.relopt.RelOptUtil;
-import org.eigenbase.util.*;
-
-import com.google.common.base.*;
+package org.apache.calcite.test;
+
+import org.apache.calcite.DataContext;
+import org.apache.calcite.adapter.clone.CloneSchema;
+import org.apache.calcite.adapter.java.ReflectiveSchema;
+import org.apache.calcite.adapter.jdbc.JdbcSchema;
+import org.apache.calcite.config.CalciteConnectionProperty;
+import org.apache.calcite.jdbc.CalciteConnection;
+import org.apache.calcite.jdbc.CalciteSchema;
+import org.apache.calcite.jdbc.MetaImpl;
+import org.apache.calcite.materialize.Lattice;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.runtime.Hook;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.schema.impl.AbstractSchema;
+import org.apache.calcite.schema.impl.ViewTable;
+import org.apache.calcite.util.JsonBuilder;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
+
 import com.google.common.base.Function;
-import com.google.common.cache.*;
+import com.google.common.base.Functions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMultiset;
 import com.google.common.collect.Lists;
 
 import java.io.PrintWriter;
 import java.io.StringWriter;
-import java.lang.reflect.*;
-import java.sql.*;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+import java.sql.Statement;
 import java.text.DateFormat;
 import java.text.SimpleDateFormat;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.TimeZone;
+import java.util.TreeSet;
 import java.util.concurrent.atomic.AtomicInteger;
 import javax.sql.DataSource;
 
-import static org.hamcrest.CoreMatchers.*;
-import static org.junit.Assert.*;
+import static org.hamcrest.CoreMatchers.containsString;
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 /**
  * Fluid DSL for testing Calcite connections and queries.
  */
-public class OptiqAssert {
-  private OptiqAssert() {}
+public class CalciteAssert {
+  private CalciteAssert() {}
 
   /** Which database to use for tests that require a JDBC data source. By
    * default the test suite runs against the embedded hsqldb database.
@@ -94,33 +120,27 @@ public class OptiqAssert {
   /** Implementation of {@link AssertThat} that does nothing. */
   private static final AssertThat DISABLED =
       new AssertThat((Config) null) {
-        @Override
-        public AssertThat with(Config config) {
+        @Override public AssertThat with(Config config) {
           return this;
         }
 
-        @Override
-        public AssertThat with(ConnectionFactory connectionFactory) {
+        @Override public AssertThat with(ConnectionFactory connectionFactory) {
           return this;
         }
 
-        @Override
-        public AssertThat with(Map<String, String> map) {
+        @Override public AssertThat with(Map<String, String> map) {
           return this;
         }
 
-        @Override
-        public AssertThat with(String name, Object schema) {
+        @Override public AssertThat with(String name, Object schema) {
           return this;
         }
 
-        @Override
-        public AssertThat withModel(String model) {
+        @Override public AssertThat withModel(String model) {
           return this;
         }
 
-        @Override
-        public AssertQuery query(String sql) {
+        @Override public AssertQuery query(String sql) {
           return NopAssertQuery.of(sql);
         }
 
@@ -129,29 +149,26 @@ public class OptiqAssert {
           return this;
         }
 
-        @Override
-        public <T> AssertThat doWithConnection(Function<OptiqConnection, T> fn)
+        @Override public <T> AssertThat doWithConnection(
+            Function<CalciteConnection, T> fn)
             throws Exception {
           return this;
         }
 
-        @Override
-        public AssertThat withSchema(String schema) {
+        @Override public AssertThat withSchema(String schema) {
           return this;
         }
 
-        @Override
-        public AssertThat enable(boolean enabled) {
+        @Override public AssertThat enable(boolean enabled) {
           return this;
         }
 
-        @Override
-        public AssertThat pooled() {
+        @Override public AssertThat pooled() {
           return this;
         }
       };
 
-  /** Creates an instance of {@code OptiqAssert} with the regular
+  /** Creates an instance of {@code CalciteAssert} with the regular
    * configuration. */
   public static AssertThat that() {
     return new AssertThat(Config.REGULAR);
@@ -192,7 +209,7 @@ public class OptiqAssert {
     return new Function<ResultSet, Void>() {
       public Void apply(ResultSet resultSet) {
         try {
-          final String resultString = OptiqAssert.toString(resultSet);
+          final String resultString = CalciteAssert.toString(resultSet);
           assertEquals(expected, Util.toLinux(resultString));
           return null;
         } catch (SQLException e) {
@@ -226,7 +243,7 @@ public class OptiqAssert {
     return new Function<ResultSet, Void>() {
       public Void apply(ResultSet resultSet) {
         try {
-          final int count = OptiqAssert.countRows(resultSet);
+          final int count = CalciteAssert.countRows(resultSet);
           assertEquals(expected, count);
           return null;
         } catch (SQLException e) {
@@ -250,7 +267,7 @@ public class OptiqAssert {
         ++executeCount;
         try {
           final Collection result =
-              OptiqAssert.toStringList(resultSet,
+              CalciteAssert.toStringList(resultSet,
                   ordered ? new ArrayList<String>() : new TreeSet<String>());
           if (executeCount == 1) {
             expected = result;
@@ -286,7 +303,7 @@ public class OptiqAssert {
           Collections.sort(expectedList);
 
           final List<String> actualList = Lists.newArrayList();
-          OptiqAssert.toStringList(resultSet, actualList);
+          CalciteAssert.toStringList(resultSet, actualList);
           Collections.sort(actualList);
 
           // Use assertArrayEquals since it implements fine-grained comparison.
@@ -304,7 +321,7 @@ public class OptiqAssert {
     return new Function<ResultSet, Void>() {
       public Void apply(ResultSet s) {
         try {
-          final String actual = Util.toLinux(OptiqAssert.toString(s));
+          final String actual = Util.toLinux(CalciteAssert.toString(s));
           if (!actual.contains(expected)) {
             assertEquals("contains", expected, actual);
           }
@@ -321,7 +338,7 @@ public class OptiqAssert {
     return new Function<ResultSet, Void>() {
       public Void apply(ResultSet s) {
         try {
-          final String actual = Util.toLinux(OptiqAssert.toString(s));
+          final String actual = Util.toLinux(CalciteAssert.toString(s));
           final String maskedActual =
               actual.replaceAll(", id = [0-9]+", "");
           if (!maskedActual.contains(expected)) {
@@ -375,14 +392,14 @@ public class OptiqAssert {
       Function<Throwable, Void> exceptionChecker) throws Exception {
     final String message =
         "With materializationsEnabled=" + materializationsEnabled
-        + ", limit=" + limit;
+            + ", limit=" + limit;
     final List<Hook.Closeable> closeableList = Lists.newArrayList();
     try {
-      ((OptiqConnection) connection).getProperties().setProperty(
-          OptiqConnectionProperty.MATERIALIZATIONS_ENABLED.camelName(),
+      ((CalciteConnection) connection).getProperties().setProperty(
+          CalciteConnectionProperty.MATERIALIZATIONS_ENABLED.camelName(),
           Boolean.toString(materializationsEnabled));
-      ((OptiqConnection) connection).getProperties().setProperty(
-          OptiqConnectionProperty.CREATE_MATERIALIZATIONS.camelName(),
+      ((CalciteConnection) connection).getProperties().setProperty(
+          CalciteConnectionProperty.CREATE_MATERIALIZATIONS.camelName(),
           Boolean.toString(materializationsEnabled));
       for (Pair<Hook, Function> hook : hooks) {
         closeableList.add(hook.left.addThread(hook.right));
@@ -453,11 +470,11 @@ public class OptiqAssert {
                   }
                 });
     try {
-      ((OptiqConnection) connection).getProperties().setProperty(
-          OptiqConnectionProperty.MATERIALIZATIONS_ENABLED.camelName(),
+      ((CalciteConnection) connection).getProperties().setProperty(
+          CalciteConnectionProperty.MATERIALIZATIONS_ENABLED.camelName(),
           Boolean.toString(materializationsEnabled));
-      ((OptiqConnection) connection).getProperties().setProperty(
-          OptiqConnectionProperty.CREATE_MATERIALIZATIONS.camelName(),
+      ((CalciteConnection) connection).getProperties().setProperty(
+          CalciteConnectionProperty.CREATE_MATERIALIZATIONS.camelName(),
           Boolean.toString(materializationsEnabled));
       PreparedStatement statement = connection.prepareStatement(sql);
       statement.close();
@@ -587,21 +604,23 @@ public class OptiqAssert {
     case JDBC_FOODMART_WITH_LATTICE:
       foodmart = rootSchema.getSubSchema("foodmart");
       if (foodmart == null) {
-        foodmart = OptiqAssert.addSchema(rootSchema, SchemaSpec.JDBC_FOODMART);
+        foodmart =
+            CalciteAssert.addSchema(rootSchema, SchemaSpec.JDBC_FOODMART);
       }
       foodmart.add("lattice",
-          Lattice.create(foodmart.unwrap(OptiqSchema.class),
+          Lattice.create(foodmart.unwrap(CalciteSchema.class),
               "select 1 from \"foodmart\".\"sales_fact_1997\" as s\n"
-              + "join \"foodmart\".\"time_by_day\" as t using (\"time_id\")\n"
-              + "join \"foodmart\".\"customer\" as c using (\"customer_id\")\n"
-              + "join \"foodmart\".\"product\" as p using (\"product_id\")\n"
-              + "join \"foodmart\".\"product_class\" as pc on p.\"product_class_id\" = pc.\"product_class_id\"",
+                  + "join \"foodmart\".\"time_by_day\" as t using (\"time_id\")\n"
+                  + "join \"foodmart\".\"customer\" as c using (\"customer_id\")\n"
+                  + "join \"foodmart\".\"product\" as p using (\"product_id\")\n"
+                  + "join \"foodmart\".\"product_class\" as pc on p.\"product_class_id\" = pc.\"product_class_id\"",
               true));
       return foodmart;
     case CLONE_FOODMART:
       foodmart = rootSchema.getSubSchema("foodmart");
       if (foodmart == null) {
-        foodmart = OptiqAssert.addSchema(rootSchema, SchemaSpec.JDBC_FOODMART);
+        foodmart =
+            CalciteAssert.addSchema(rootSchema, SchemaSpec.JDBC_FOODMART);
       }
       return rootSchema.add("foodmart2", new CloneSchema(foodmart));
     case HR:
@@ -615,34 +634,34 @@ public class OptiqAssert {
       post.add("EMP",
           ViewTable.viewMacro(post,
               "select * from (values\n"
-              + "    ('Jane', 10, 'F'),\n"
-              + "    ('Bob', 10, 'M'),\n"
-              + "    ('Eric', 20, 'M'),\n"
-              + "    ('Susan', 30, 'F'),\n"
-              + "    ('Alice', 30, 'F'),\n"
-              + "    ('Adam', 50, 'M'),\n"
-              + "    ('Eve', 50, 'F'),\n"
-              + "    ('Grace', 60, 'F'),\n"
-              + "    ('Wilma', cast(null as integer), 'F'))\n"
-              + "  as t(ename, deptno, gender)",
+                  + "    ('Jane', 10, 'F'),\n"
+                  + "    ('Bob', 10, 'M'),\n"
+                  + "    ('Eric', 20, 'M'),\n"
+                  + "    ('Susan', 30, 'F'),\n"
+                  + "    ('Alice', 30, 'F'),\n"
+                  + "    ('Adam', 50, 'M'),\n"
+                  + "    ('Eve', 50, 'F'),\n"
+                  + "    ('Grace', 60, 'F'),\n"
+                  + "    ('Wilma', cast(null as integer), 'F'))\n"
+                  + "  as t(ename, deptno, gender)",
               ImmutableList.<String>of()));
       post.add("DEPT",
           ViewTable.viewMacro(post,
               "select * from (values\n"
-              + "    (10, 'Sales'),\n"
-              + "    (20, 'Marketing'),\n"
-              + "    (30, 'Engineering'),\n"
-              + "    (40, 'Empty')) as t(deptno, dname)",
+                  + "    (10, 'Sales'),\n"
+                  + "    (20, 'Marketing'),\n"
+                  + "    (30, 'Engineering'),\n"
+                  + "    (40, 'Empty')) as t(deptno, dname)",
               ImmutableList.<String>of()));
       post.add("EMPS",
           ViewTable.viewMacro(post,
               "select * from (values\n"
-              + "    (100, 'Fred',  10, CAST(NULL AS CHAR(1)), CAST(NULL AS VARCHAR(20)), 40,               25, TRUE,    FALSE, DATE '1996-08-03'),\n"
-              + "    (110, 'Eric',  20, 'M',                   'San Francisco',           3,                80, UNKNOWN, FALSE, DATE '2001-01-01'),\n"
-              + "    (110, 'John',  40, 'M',                   'Vancouver',               2, CAST(NULL AS INT), FALSE,   TRUE,  DATE '2002-05-03'),\n"
-              + "    (120, 'Wilma', 20, 'F',                   CAST(NULL AS VARCHAR(20)), 1,                 5, UNKNOWN, TRUE,  DATE '2005-09-07'),\n"
-              + "    (130, 'Alice', 40, 'F',                   'Vancouver',               2, CAST(NULL AS INT), FALSE,   TRUE,  DATE '2007-01-01'))\n"
-              + " as t(empno, name, deptno, gender, city, empid, age, slacker, manager, joinedat)",
+                  + "    (100, 'Fred',  10, CAST(NULL AS CHAR(1)), CAST(NULL AS VARCHAR(20)), 40,               25, TRUE,    FALSE, DATE '1996-08-03'),\n"
+                  + "    (110, 'Eric',  20, 'M',                   'San Francisco',           3,                80, UNKNOWN, FALSE, DATE '2001-01-01'),\n"
+                  + "    (110, 'John',  40, 'M',                   'Vancouver',               2, CAST(NULL AS INT), FALSE,   TRUE,  DATE '2002-05-03'),\n"
+                  + "    (120, 'Wilma', 20, 'F',                   CAST(NULL AS VARCHAR(20)), 1,                 5, UNKNOWN, TRUE,  DATE '2005-09-07'),\n"
+                  + "    (130, 'Alice', 40, 'F',                   'Vancouver',               2, CAST(NULL AS INT), FALSE,   TRUE,  DATE '2007-01-01'))\n"
+                  + " as t(empno, name, deptno, gender, city, empid, age, slacker, manager, joinedat)",
               ImmutableList.<String>of()));
       return post;
     default:
@@ -650,16 +669,15 @@ public class OptiqAssert {
     }
   }
 
-  static OptiqConnection getConnection(String... schema)
+  static CalciteConnection getConnection(String... schema)
       throws ClassNotFoundException, SQLException {
     final List<String> schemaList = Arrays.asList(schema);
-    Class.forName("net.hydromatic.optiq.jdbc.Driver");
     String suffix = schemaList.contains("spark") ? "spark=true" : "";
     Connection connection =
         DriverManager.getConnection("jdbc:calcite:" + suffix);
-    OptiqConnection optiqConnection =
-        connection.unwrap(OptiqConnection.class);
-    SchemaPlus rootSchema = optiqConnection.getRootSchema();
+    CalciteConnection calciteConnection =
+        connection.unwrap(CalciteConnection.class);
+    SchemaPlus rootSchema = calciteConnection.getRootSchema();
     if (schemaList.contains("hr")) {
       addSchema(rootSchema, SchemaSpec.HR);
     }
@@ -676,7 +694,7 @@ public class OptiqAssert {
       // always present
       Util.discard(0);
     }
-    return optiqConnection;
+    return calciteConnection;
   }
 
   /**
@@ -690,13 +708,12 @@ public class OptiqAssert {
    * @throws ClassNotFoundException
    * @throws java.sql.SQLException
    */
-  static OptiqConnection getConnection(SchemaSpec schemaSpec)
+  static CalciteConnection getConnection(SchemaSpec schemaSpec)
       throws ClassNotFoundException, SQLException {
-    Class.forName("net.hydromatic.optiq.jdbc.Driver");
     Connection connection = DriverManager.getConnection("jdbc:calcite:");
-    OptiqConnection optiqConnection =
-        connection.unwrap(OptiqConnection.class);
-    final SchemaPlus rootSchema = optiqConnection.getRootSchema();
+    CalciteConnection calciteConnection =
+        connection.unwrap(CalciteConnection.class);
+    final SchemaPlus rootSchema = calciteConnection.getRootSchema();
     switch (schemaSpec) {
     case JDBC_FOODMART:
       addSchema(rootSchema, schemaSpec);
@@ -709,8 +726,8 @@ public class OptiqAssert {
     default:
       throw new AssertionError("unknown schema " + schemaSpec);
     }
-    optiqConnection.setSchema("foodmart2");
-    return optiqConnection;
+    calciteConnection.setSchema("foodmart2");
+    return calciteConnection;
   }
 
   static <F, T> Function<F, T> constantNull() {
@@ -719,7 +736,7 @@ public class OptiqAssert {
   }
 
   /**
-   * Result of calling {@link OptiqAssert#that}.
+   * Result of calling {@link CalciteAssert#that}.
    */
   public static class AssertThat {
     private final ConnectionFactory connectionFactory;
@@ -743,13 +760,12 @@ public class OptiqAssert {
     public AssertThat with(final Map<String, String> map) {
       return new AssertThat(
           new ConnectionFactory() {
-            public OptiqConnection createConnection() throws Exception {
-              Class.forName("net.hydromatic.optiq.jdbc.Driver");
+            public CalciteConnection createConnection() throws Exception {
               final Properties info = new Properties();
               for (Map.Entry<String, String> entry : map.entrySet()) {
                 info.setProperty(entry.getKey(), entry.getValue());
               }
-              return (OptiqConnection) DriverManager.getConnection(
+              return (CalciteConnection) DriverManager.getConnection(
                   "jdbc:calcite:", info);
             }
           });
@@ -759,30 +775,28 @@ public class OptiqAssert {
      * object. */
     public AssertThat with(final String name, final Object schema) {
       return with(
-          new OptiqAssert.ConnectionFactory() {
-            public OptiqConnection createConnection() throws Exception {
-              Class.forName("net.hydromatic.optiq.jdbc.Driver");
+          new CalciteAssert.ConnectionFactory() {
+            public CalciteConnection createConnection() throws Exception {
               Connection connection =
                   DriverManager.getConnection("jdbc:calcite:");
-              OptiqConnection optiqConnection =
-                  connection.unwrap(OptiqConnection.class);
+              CalciteConnection calciteConnection =
+                  connection.unwrap(CalciteConnection.class);
               SchemaPlus rootSchema =
-                  optiqConnection.getRootSchema();
+                  calciteConnection.getRootSchema();
               rootSchema.add(name, new ReflectiveSchema(schema));
-              optiqConnection.setSchema(name);
-              return optiqConnection;
+              calciteConnection.setSchema(name);
+              return calciteConnection;
             }
           });
     }
 
     public AssertThat withModel(final String model) {
       return new AssertThat(
-          new OptiqAssert.ConnectionFactory() {
-            public OptiqConnection createConnection() throws Exception {
-              Class.forName("net.hydromatic.optiq.jdbc.Driver");
+          new CalciteAssert.ConnectionFactory() {
+            public CalciteConnection createConnection() throws Exception {
               final Properties info = new Properties();
               info.setProperty("model", "inline:" + model);
-              return (OptiqConnection) DriverManager.getConnection(
+              return (CalciteConnection) DriverManager.getConnection(
                   "jdbc:calcite:", info);
             }
           });
@@ -854,12 +868,13 @@ public class OptiqAssert {
       return this;
     }
 
-    /** Creates a {@link OptiqConnection} and executes a callback. */
-    public <T> AssertThat doWithConnection(Function<OptiqConnection, T> fn)
+    /** Creates a {@link org.apache.calcite.jdbc.CalciteConnection}
+     * and executes a callback. */
+    public <T> AssertThat doWithConnection(Function<CalciteConnection, T> fn)
         throws Exception {
       Connection connection = connectionFactory.createConnection();
       try {
-        T t = fn.apply((OptiqConnection) connection);
+        T t = fn.apply((CalciteConnection) connection);
         Util.discard(t);
         return AssertThat.this;
       } finally {
@@ -870,7 +885,7 @@ public class OptiqAssert {
     /** Creates a {@link DataContext} and executes a callback. */
     public <T> AssertThat doWithDataContext(Function<DataContext, T> fn)
         throws Exception {
-      OptiqConnection connection = connectionFactory.createConnection();
+      CalciteConnection connection = connectionFactory.createConnection();
       final DataContext dataContext = MetaImpl.createDataContext(connection);
       try {
         T t = fn.apply(dataContext);
@@ -906,10 +921,12 @@ public class OptiqAssert {
     }
   }
 
+  /** Connection factory. */
   public interface ConnectionFactory {
-    OptiqConnection createConnection() throws Exception;
+    CalciteConnection createConnection() throws Exception;
   }
 
+  /** Connection factory that uses the same instance of connections. */
   private static class PoolingConnectionFactory implements ConnectionFactory {
     private final ConnectionFactory factory;
 
@@ -917,24 +934,27 @@ public class OptiqAssert {
       this.factory = factory;
     }
 
-    public OptiqConnection createConnection() throws Exception {
+    public CalciteConnection createConnection() throws Exception {
       return Pool.INSTANCE.cache.get(factory);
     }
   }
 
+  /** Connection pool. */
   private static class Pool {
     private static final Pool INSTANCE = new Pool();
 
-    private final LoadingCache<ConnectionFactory, OptiqConnection> cache =
+    private final LoadingCache<ConnectionFactory, CalciteConnection> cache =
         CacheBuilder.newBuilder().build(
-            new CacheLoader<ConnectionFactory, OptiqConnection>() {
-              public OptiqConnection load(ConnectionFactory key)
+            new CacheLoader<ConnectionFactory, CalciteConnection>() {
+              public CalciteConnection load(ConnectionFactory key)
                   throws Exception {
                 return key.createConnection();
               }
             });
   }
 
+  /** Connection factory that creates connections based on a given
+   * {@link Config}. */
   private static class ConfigConnectionFactory implements ConnectionFactory {
     private final Config config;
 
@@ -952,7 +972,7 @@ public class OptiqAssert {
           && config == ((ConfigConnectionFactory) obj).config;
     }
 
-    public OptiqConnection createConnection() throws Exception {
+    public CalciteConnection createConnection() throws Exception {
       switch (config) {
       case REGULAR:
         return getConnection("hr", "foodmart", "post");
@@ -961,7 +981,7 @@ public class OptiqAssert {
       case LINGUAL:
         return getConnection("lingual");
       case JDBC_FOODMART:
-        return getConnection(OptiqAssert.SchemaSpec.JDBC_FOODMART);
+        return getConnection(CalciteAssert.SchemaSpec.JDBC_FOODMART);
       case FOODMART_CLONE:
         return getConnection(SchemaSpec.CLONE_FOODMART);
       case JDBC_FOODMART_WITH_LATTICE:
@@ -974,6 +994,7 @@ public class OptiqAssert {
     }
   }
 
+  /** Connection factory that delegates to an underlying factory. */
   private static class DelegatingConnectionFactory
       implements ConnectionFactory {
     private final ConnectionFactory factory;
@@ -982,11 +1003,13 @@ public class OptiqAssert {
       this.factory = factory;
     }
 
-    public OptiqConnection createConnection() throws Exception {
+    public CalciteConnection createConnection() throws Exception {
       return factory.createConnection();
     }
   }
 
+  /** Connection factory that gets a connection from an underlying factory, then
+   * sets its schema. */
   private static class SchemaConnectionFactory
       extends DelegatingConnectionFactory {
     private final String schema;
@@ -996,14 +1019,14 @@ public class OptiqAssert {
       this.schema = schema;
     }
 
-    @Override
-    public OptiqConnection createConnection() throws Exception {
-      OptiqConnection connection = super.createConnection();
+    @Override public CalciteConnection createConnection() throws Exception {
+      CalciteConnection connection = super.createConnection();
       connection.setSchema(schema);
       return connection;
     }
   }
 
+  /** Fluent interface for building a query to be tested. */
   public static class AssertQuery {
     private final String sql;
     private ConnectionFactory connectionFactory;
@@ -1231,11 +1254,13 @@ public class OptiqAssert {
     }
   }
 
+  /** Connection configuration. Basically, a set of schemas that should be
+   * instantiated in the connection. */
   public enum Config {
     /**
      * Configuration that creates a connection with two in-memory data sets:
-     * {@link net.hydromatic.optiq.test.JdbcTest.HrSchema} and
-     * {@link net.hydromatic.optiq.test.JdbcTest.FoodmartSchema}.
+     * {@link org.apache.calcite.test.JdbcTest.HrSchema} and
+     * {@link org.apache.calcite.test.JdbcTest.FoodmartSchema}.
      */
     REGULAR,
 
@@ -1250,7 +1275,7 @@ public class OptiqAssert {
      * such as "customer" and "sales_fact_1997" are available. Queries
      * are processed by generating Java that calls linq4j operators
      * such as
-     * {@link net.hydromatic.linq4j.Enumerable#where(net.hydromatic.linq4j.function.Predicate1)}.
+     * {@link org.apache.calcite.linq4j.Enumerable#where(org.apache.calcite.linq4j.function.Predicate1)}.
      */
     JDBC_FOODMART,
 
@@ -1280,23 +1305,20 @@ public class OptiqAssert {
       return new NopAssertQuery(sql);
     }
 
-    @Override
-    protected Connection createConnection() throws Exception {
+    @Override protected Connection createConnection() throws Exception {
       throw new AssertionError("disabled");
     }
 
-    @Override
-    public AssertQuery returns(String sql, Function<ResultSet, Void> checker) {
+    @Override public AssertQuery returns(String sql,
+        Function<ResultSet, Void> checker) {
       return this;
     }
 
-    @Override
-    public AssertQuery throws_(String message) {
+    @Override public AssertQuery throws_(String message) {
       return this;
     }
 
-    @Override
-    public AssertQuery runs() {
+    @Override public AssertQuery runs() {
       return this;
     }
 
@@ -1310,18 +1332,16 @@ public class OptiqAssert {
       return this;
     }
 
-    @Override
-    public AssertQuery planContains(String expected) {
+    @Override public AssertQuery planContains(String expected) {
       return this;
     }
 
-    @Override
-    public AssertQuery planHasSql(String expected) {
+    @Override public AssertQuery planHasSql(String expected) {
       return this;
     }
 
-    @Override
-    public AssertQuery queryContains(Function<List, Void> predicate1) {
+    @Override public AssertQuery
+    queryContains(Function<List, Void> predicate1) {
       return this;
     }
   }
@@ -1348,6 +1368,7 @@ public class OptiqAssert {
     }
   }
 
+  /** Specification for common test schemas. */
   public enum SchemaSpec {
     REFLECTIVE_FOODMART,
     JDBC_FOODMART,
@@ -1359,4 +1380,4 @@ public class OptiqAssert {
   }
 }
 
-// End OptiqAssert.java
+// End CalciteAssert.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/test/CalciteResourceTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/CalciteResourceTest.java b/core/src/test/java/org/apache/calcite/test/CalciteResourceTest.java
index fb81edb..7bf4eae 100644
--- a/core/src/test/java/org/apache/calcite/test/CalciteResourceTest.java
+++ b/core/src/test/java/org/apache/calcite/test/CalciteResourceTest.java
@@ -14,25 +14,26 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.test;
-
-import java.util.*;
+package org.apache.calcite.test;
 
 import org.hamcrest.CoreMatchers;
 import org.junit.Test;
 
-import static org.eigenbase.util.Static.RESOURCE;
+import java.util.Map;
+
+import static org.apache.calcite.util.Static.RESOURCE;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertThat;
 
 /**
- * Tests generated package org.eigenbase.resource (mostly a sanity check for
- * resgen infrastructure).
+ * Tests the generated implementation of
+ * {@link org.apache.calcite.runtime.CalciteResource} (mostly a sanity check for
+ * the resource-generation infrastructure).
  */
-public class EigenbaseResourceTest {
+public class CalciteResourceTest {
   //~ Constructors -----------------------------------------------------------
 
-  public EigenbaseResourceTest() {
+  public CalciteResourceTest() {
   }
 
   //~ Methods ----------------------------------------------------------------
@@ -48,4 +49,4 @@ public class EigenbaseResourceTest {
   }
 }
 
-// End EigenbaseResourceTest.java
+// End CalciteResourceTest.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/test/CalciteSqlOperatorTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/CalciteSqlOperatorTest.java b/core/src/test/java/org/apache/calcite/test/CalciteSqlOperatorTest.java
index ecdce14..dc87ab8 100644
--- a/core/src/test/java/org/apache/calcite/test/CalciteSqlOperatorTest.java
+++ b/core/src/test/java/org/apache/calcite/test/CalciteSqlOperatorTest.java
@@ -14,23 +14,22 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.test;
+package org.apache.calcite.test;
 
-import net.hydromatic.optiq.jdbc.OptiqConnection;
-
-import org.eigenbase.sql.test.SqlOperatorBaseTest;
-import org.eigenbase.sql.test.SqlTester;
+import org.apache.calcite.jdbc.CalciteConnection;
+import org.apache.calcite.sql.test.SqlOperatorBaseTest;
+import org.apache.calcite.sql.test.SqlTester;
 
 /**
- * Embodiment of {@link org.eigenbase.sql.test.SqlOperatorBaseTest}
+ * Embodiment of {@link org.apache.calcite.sql.test.SqlOperatorBaseTest}
  * that generates SQL statements and executes them using Calcite.
  */
-public class OptiqSqlOperatorTest extends SqlOperatorBaseTest {
-  private static final ThreadLocal<OptiqConnection> LOCAL =
-      new ThreadLocal<OptiqConnection>() {
-        @Override protected OptiqConnection initialValue() {
+public class CalciteSqlOperatorTest extends SqlOperatorBaseTest {
+  private static final ThreadLocal<CalciteConnection> LOCAL =
+      new ThreadLocal<CalciteConnection>() {
+        @Override protected CalciteConnection initialValue() {
           try {
-            return OptiqAssert.getConnection("hr");
+            return CalciteAssert.getConnection("hr");
           } catch (Exception e) {
             throw new RuntimeException(e);
           }
@@ -41,9 +40,9 @@ public class OptiqSqlOperatorTest extends SqlOperatorBaseTest {
     return tester(LOCAL.get());
   }
 
-  public OptiqSqlOperatorTest() {
+  public CalciteSqlOperatorTest() {
     super(false, getHrTester());
   }
 }
 
-// End OptiqSqlOperatorTest.java
+// End CalciteSqlOperatorTest.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/test/CalciteSuite.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/CalciteSuite.java b/core/src/test/java/org/apache/calcite/test/CalciteSuite.java
index 8e0dd1a..3a3ec53 100644
--- a/core/src/test/java/org/apache/calcite/test/CalciteSuite.java
+++ b/core/src/test/java/org/apache/calcite/test/CalciteSuite.java
@@ -14,27 +14,31 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.test;
+package org.apache.calcite.test;
 
-import net.hydromatic.optiq.impl.clone.ArrayTableTest;
-import net.hydromatic.optiq.runtime.BinarySearchTest;
-import net.hydromatic.optiq.runtime.EnumerablesTest;
-import net.hydromatic.optiq.tools.FrameworksTest;
-import net.hydromatic.optiq.tools.PlannerTest;
-import net.hydromatic.optiq.util.BitSetsTest;
-import net.hydromatic.optiq.util.PartiallyOrderedSetTest;
-import net.hydromatic.optiq.util.graph.DirectedGraphTest;
-
-import org.eigenbase.relopt.RelOptUtilTest;
-import org.eigenbase.relopt.RelWriterTest;
-import org.eigenbase.relopt.volcano.VolcanoPlannerTest;
-import org.eigenbase.relopt.volcano.VolcanoPlannerTraitTest;
-import org.eigenbase.rex.RexExecutorTest;
-import org.eigenbase.sql.parser.SqlParserTest;
-import org.eigenbase.sql.test.*;
-import org.eigenbase.test.*;
-import org.eigenbase.util.*;
-import org.eigenbase.util.mapping.MappingTest;
+import org.apache.calcite.adapter.clone.ArrayTableTest;
+import org.apache.calcite.plan.RelOptUtilTest;
+import org.apache.calcite.plan.RelWriterTest;
+import org.apache.calcite.plan.volcano.VolcanoPlannerTest;
+import org.apache.calcite.plan.volcano.VolcanoPlannerTraitTest;
+import org.apache.calcite.rex.RexExecutorTest;
+import org.apache.calcite.runtime.BinarySearchTest;
+import org.apache.calcite.runtime.EnumerablesTest;
+import org.apache.calcite.sql.parser.SqlParserTest;
+import org.apache.calcite.sql.test.SqlAdvisorTest;
+import org.apache.calcite.sql.test.SqlOperatorTest;
+import org.apache.calcite.sql.test.SqlPrettyWriterTest;
+import org.apache.calcite.sql.test.SqlTypeNameTest;
+import org.apache.calcite.tools.FrameworksTest;
+import org.apache.calcite.tools.PlannerTest;
+import org.apache.calcite.util.BitSetsTest;
+import org.apache.calcite.util.ChunkListTest;
+import org.apache.calcite.util.PartiallyOrderedSetTest;
+import org.apache.calcite.util.PermutationTestCase;
+import org.apache.calcite.util.ReflectVisitorTest;
+import org.apache.calcite.util.UtilTest;
+import org.apache.calcite.util.graph.DirectedGraphTest;
+import org.apache.calcite.util.mapping.MappingTest;
 
 import org.junit.runner.RunWith;
 import org.junit.runners.Suite;
@@ -58,7 +62,7 @@ import org.junit.runners.Suite;
     RelOptUtilTest.class,
     UtilTest.class,
     MappingTest.class,
-    EigenbaseResourceTest.class,
+    CalciteResourceTest.class,
     FilteratorTest.class,
     PermutationTestCase.class,
     SqlFunctionsTest.class,
@@ -69,7 +73,6 @@ import org.junit.runners.Suite;
     InterpreterTest.class,
     VolcanoPlannerTest.class,
     HepPlannerTest.class,
-    SargTest.class,
     RelWriterTest.class,
     RexProgramTest.class,
     RexTransformerTest.class,
@@ -100,7 +103,7 @@ import org.junit.runners.Suite;
     JdbcAdapterTest.class,
     LinqFrontJdbcBackTest.class,
     JdbcFrontJdbcBackLinqMiddleTest.class,
-    OptiqSqlOperatorTest.class,
+    CalciteSqlOperatorTest.class,
     LatticeTest.class,
     ReflectiveSchemaTest.class,
     JdbcTest.class,
@@ -117,7 +120,7 @@ import org.junit.runners.Suite;
     // '-Dcalcite.test.slow=true' is specified)
     FoodmartTest.class
 })
-public class OptiqSuite {
+public class CalciteSuite {
 }
 
-// End OptiqSuite.java
+// End CalciteSuite.java


[38/58] [abbrv] [partial] incubator-calcite git commit: [CALCITE-306] Standardize code style for "import package.*; "

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 6cd21f2..4a60299 100644
--- a/core/src/main/java/org/apache/calcite/rel/AbstractRelNode.java
+++ b/core/src/main/java/org/apache/calcite/rel/AbstractRelNode.java
@@ -14,24 +14,41 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel;
-
-import java.io.*;
-import java.util.*;
-import java.util.logging.*;
-
-import org.eigenbase.rel.metadata.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.rex.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.trace.*;
-import org.eigenbase.util.*;
-
-import net.hydromatic.optiq.util.BitSets;
+package org.apache.calcite.rel;
+
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.ConventionTraitDef;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelOptQuery;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.RelTrait;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.externalize.RelWriterImpl;
+import org.apache.calcite.rel.metadata.Metadata;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
+import org.apache.calcite.util.trace.CalciteTrace;
 
 import com.google.common.collect.ImmutableList;
 
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
 /**
  * Base class for every relational expression ({@link RelNode}).
  */
@@ -44,7 +61,7 @@ public abstract class AbstractRelNode implements RelNode {
   /** Generator for {@link #id} values. */
   static int nextId = 0;
 
-  private static final Logger LOGGER = EigenbaseTrace.getPlannerTracer();
+  private static final Logger LOGGER = CalciteTrace.getPlannerTracer();
 
   //~ Instance fields --------------------------------------------------------
 
@@ -61,9 +78,9 @@ public abstract class AbstractRelNode implements RelNode {
   /**
    * A short description of this relational expression's type, inputs, and
    * other properties. The string uniquely identifies the node; another node
-   * is equivalent if and only if it has the same value. Computed by {@link
-   * #computeDigest}, assigned by {@link #onRegister}, returned by {@link
-   * #getDigest()}.
+   * is equivalent if and only if it has the same value. Computed by
+   * {@link #computeDigest}, assigned by {@link #onRegister}, returned by
+   * {@link #getDigest()}.
    *
    * @see #desc
    */
@@ -115,9 +132,10 @@ public abstract class AbstractRelNode implements RelNode {
         && traitSet == getTraitSet()) {
       return this;
     }
-    throw new AssertionError(
-        "Relational expression should override copy. Class=[" + getClass()
-        + "]; traits=[" + getTraitSet() + "]; desired traits=[" + traitSet
+    throw new AssertionError("Relational expression should override copy. "
+        + "Class=[" + getClass()
+        + "]; traits=[" + getTraitSet()
+        + "]; desired traits=[" + traitSet
         + "]");
   }
 
@@ -252,8 +270,8 @@ public abstract class AbstractRelNode implements RelNode {
   }
 
   public RelNode accept(RelShuttle shuttle) {
-    // Call fall-back method. Specific logical types (such as ProjectRel
-    // and JoinRel) have their own RelShuttle.visit methods.
+    // Call fall-back method. Specific logical types (such as LogicalProject
+    // and LogicalJoin) have their own RelShuttle.visit methods.
     return shuttle.visit(this);
   }
 
@@ -283,9 +301,10 @@ public abstract class AbstractRelNode implements RelNode {
   /**
    * Describes the inputs and attributes of this relational expression.
    * Each node should call {@code super.explainTerms}, then call the
-   * {@link RelWriterImpl#input(String, RelNode)}
-   * and {@link RelWriterImpl#item(String, Object)} methods for each input
-   * and attribute.
+   * {@link org.apache.calcite.rel.externalize.RelWriterImpl#input(String, RelNode)}
+   * and
+   * {@link org.apache.calcite.rel.externalize.RelWriterImpl#item(String, Object)}
+   * methods for each input and attribute.
    *
    * @param pw Plan writer
    */

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/Aggregation.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/Aggregation.java b/core/src/main/java/org/apache/calcite/rel/Aggregation.java
deleted file mode 100644
index 2af9369..0000000
--- a/core/src/main/java/org/apache/calcite/rel/Aggregation.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to you under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.eigenbase.rel;
-
-import java.util.List;
-
-import org.eigenbase.reltype.*;
-
-/**
- * An <code>Aggregation</code> aggregates a set of values into one value.
- *
- * <p>It is used, via a {@link AggregateCall}, in an {@link AggregateRel}
- * relational operator.</p>
- *
- * @deprecated Use {@link org.eigenbase.sql.SqlAggFunction};
- * after {@link org.eigenbase.util.Bug#upgrade calcite-0.9.1},
- * {@link org.eigenbase.rel.AggregateCall} will require a {@code SqlAggFunction}
- * and after {@link org.eigenbase.util.Bug#upgrade calcite-0.9.2}
- * this interface will be removed.
- */
-public interface Aggregation {
-  //~ Methods ----------------------------------------------------------------
-
-  /**
-   * Returns the parameter types accepted by this Aggregation.
-   *
-   * @param typeFactory Type factory to create the types
-   * @return Array of parameter types
-   *
-   * @deprecated Use
-   * {@link org.eigenbase.sql.SqlAggFunction#getOperandTypeInference()}; will
-   * be removed after {@link org.eigenbase.util.Bug#upgrade calcite-0.9.2}.
-   */
-  List<RelDataType> getParameterTypes(RelDataTypeFactory typeFactory);
-
-  /**
-   * Returns the type of the result yielded by this Aggregation.
-   *
-   * @param typeFactory Type factory to create the type
-   * @return Result type
-   *
-   * @deprecated Use
-   * {@link org.eigenbase.sql.SqlAggFunction#getReturnTypeInference()}; will
-   * be removed after {@link org.eigenbase.util.Bug#upgrade calcite-0.9.2}.
-   */
-  RelDataType getReturnType(RelDataTypeFactory typeFactory);
-
-  /**
-   * Returns the name of this Aggregation
-   *
-   * @return name of this aggregation
-   */
-  String getName();
-}
-
-// End Aggregation.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/InvalidRelException.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/InvalidRelException.java b/core/src/main/java/org/apache/calcite/rel/InvalidRelException.java
index e29b093..0852967 100644
--- a/core/src/main/java/org/apache/calcite/rel/InvalidRelException.java
+++ b/core/src/main/java/org/apache/calcite/rel/InvalidRelException.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel;
+package org.apache.calcite.rel;
 
 /**
  * Exception that indicates that a relational expression would be invalid

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/RelCollation.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/RelCollation.java b/core/src/main/java/org/apache/calcite/rel/RelCollation.java
index 5b80ac9..3aed2e9 100644
--- a/core/src/main/java/org/apache/calcite/rel/RelCollation.java
+++ b/core/src/main/java/org/apache/calcite/rel/RelCollation.java
@@ -14,11 +14,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel;
+package org.apache.calcite.rel;
 
-import java.util.*;
+import org.apache.calcite.plan.RelTrait;
 
-import org.eigenbase.relopt.RelTrait;
+import java.util.List;
 
 /**
  * Description of the physical ordering of a relational expression.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/RelCollationImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/RelCollationImpl.java b/core/src/main/java/org/apache/calcite/rel/RelCollationImpl.java
index 218529e..363c190 100644
--- a/core/src/main/java/org/apache/calcite/rel/RelCollationImpl.java
+++ b/core/src/main/java/org/apache/calcite/rel/RelCollationImpl.java
@@ -14,20 +14,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel;
+package org.apache.calcite.rel;
 
-import java.util.*;
-
-import org.eigenbase.relopt.RelOptPlanner;
-import org.eigenbase.relopt.RelTrait;
-import org.eigenbase.relopt.RelTraitDef;
-import org.eigenbase.reltype.*;
-import org.eigenbase.util.Util;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTrait;
+import org.apache.calcite.plan.RelTraitDef;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.util.Util;
 
 import com.google.common.base.Function;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 
+import java.util.Iterator;
+import java.util.List;
+
 /**
  * Simple implementation of {@link RelCollation}.
  */

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/RelCollationTraitDef.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/RelCollationTraitDef.java b/core/src/main/java/org/apache/calcite/rel/RelCollationTraitDef.java
index 9c918a8..dc935fd 100644
--- a/core/src/main/java/org/apache/calcite/rel/RelCollationTraitDef.java
+++ b/core/src/main/java/org/apache/calcite/rel/RelCollationTraitDef.java
@@ -14,20 +14,24 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel;
+package org.apache.calcite.rel;
 
-import org.eigenbase.relopt.*;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTraitDef;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.core.Sort;
 
 /**
  * Definition of the ordering trait.
  *
  * <p>Ordering is a physical property (i.e. a trait) because it can be changed
- * without loss of information. The converter to do this is the {@link SortRel}
- * operator.</p>
+ * without loss of information. The converter to do this is the
+ * {@link org.apache.calcite.rel.core.Sort} operator.
  *
  * <p>Unlike other current traits, a {@link RelNode} can have more than one
  * value of this trait simultaneously. For example,
- * <code>TableAccessRel(table=TIME_BY_DAY)</code> might be sorted by
+ * <code>LogicalTableScan(table=TIME_BY_DAY)</code> might be sorted by
  * <code>{the_year, the_month, the_date}</code> and also by
  * <code>{time_id}</code>. We have to allow a RelNode to belong to more than
  * one RelSubset (these RelSubsets are always in the same set).</p>
@@ -47,8 +51,7 @@ public class RelCollationTraitDef extends RelTraitDef<RelCollation> {
     return "sort";
   }
 
-  @Override
-  public boolean multiple() {
+  @Override public boolean multiple() {
     return true;
   }
 
@@ -73,8 +76,8 @@ public class RelCollationTraitDef extends RelTraitDef<RelCollation> {
     // Create a logical sort, then ask the planner to convert its remaining
     // traits (e.g. convert it to an EnumerableSortRel if rel is enumerable
     // convention)
-    final SortRel sort =
-        new SortRel(
+    final Sort sort =
+        new Sort(
             rel.getCluster(),
             rel.getCluster().traitSetOf(Convention.NONE, toCollation),
             rel,

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/RelFieldCollation.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/RelFieldCollation.java b/core/src/main/java/org/apache/calcite/rel/RelFieldCollation.java
index 0e3593f..dba73f8 100644
--- a/core/src/main/java/org/apache/calcite/rel/RelFieldCollation.java
+++ b/core/src/main/java/org/apache/calcite/rel/RelFieldCollation.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel;
+package org.apache.calcite.rel;
 
 /**
  * Definition of the ordering of one field of a {@link RelNode} whose

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/RelImplementorImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/RelImplementorImpl.java b/core/src/main/java/org/apache/calcite/rel/RelImplementorImpl.java
index e871f00..8e42bed 100644
--- a/core/src/main/java/org/apache/calcite/rel/RelImplementorImpl.java
+++ b/core/src/main/java/org/apache/calcite/rel/RelImplementorImpl.java
@@ -14,7 +14,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel;
+package org.apache.calcite.rel;
+
+import org.apache.calcite.plan.RelImplementor;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.util.trace.CalciteTrace;
 
 import java.util.ArrayList;
 import java.util.HashMap;
@@ -22,17 +28,12 @@ import java.util.List;
 import java.util.Map;
 import java.util.logging.Logger;
 
-import org.eigenbase.relopt.RelImplementor;
-import org.eigenbase.reltype.RelDataTypeFactory;
-import org.eigenbase.rex.RexBuilder;
-import org.eigenbase.trace.EigenbaseTrace;
-
 /**
  * Implementation of {@link RelImplementor}.
  */
 public class RelImplementorImpl implements RelImplementor {
   protected static final Logger LOGGER =
-      EigenbaseTrace.getRelImplementorTracer();
+      CalciteTrace.getRelImplementorTracer();
 
   /**
    * Maps a {@link String} to the {@link RelImplementorImpl.Frame} whose
@@ -42,7 +43,7 @@ public class RelImplementorImpl implements RelImplementor {
       new HashMap<String, Frame>();
 
   /**
-   * Maps a {@link org.eigenbase.rel.RelNode} to the unique frame whose
+   * Maps a {@link RelNode} to the unique frame whose
    * {@link RelImplementorImpl.Frame#rel} is
    * that relational expression.
    */
@@ -105,7 +106,7 @@ public class RelImplementorImpl implements RelImplementor {
       RelNode rel,
       int offset,
       int[] offsets) {
-    if (rel instanceof JoinRelBase) {
+    if (rel instanceof Join) {
       // no variable here -- go deeper
       List<RelNode> inputs = rel.getInputs();
       for (RelNode input : inputs) {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/RelInput.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/RelInput.java b/core/src/main/java/org/apache/calcite/rel/RelInput.java
index 963a0eb..0eff922 100644
--- a/core/src/main/java/org/apache/calcite/rel/RelInput.java
+++ b/core/src/main/java/org/apache/calcite/rel/RelInput.java
@@ -14,16 +14,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel;
+package org.apache.calcite.rel;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
 
 import java.util.BitSet;
 import java.util.List;
 
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.rex.RexLiteral;
-import org.eigenbase.rex.RexNode;
-
 /**
  * Context from which a relational expression can initialize itself,
  * reading from a serialized form of the relational expression.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 b896790..50da558 100644
--- a/core/src/main/java/org/apache/calcite/rel/RelNode.java
+++ b/core/src/main/java/org/apache/calcite/rel/RelNode.java
@@ -14,27 +14,39 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel;
-
-import java.util.*;
-
-import org.eigenbase.rel.metadata.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.rex.*;
+package org.apache.calcite.rel;
+
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptNode;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelOptQuery;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.metadata.Metadata;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexNode;
+
+import java.util.BitSet;
+import java.util.List;
+import java.util.Set;
 
 /**
  * A <code>RelNode</code> is a relational expression.
  *
+ * <p>Relational expressions process data, so their names are typically verbs:
+ * Sort, Join, Project, Filter, Scan, Sample.
+ *
  * <p>A relational expression is not a scalar expression; see
- * {@link org.eigenbase.sql.SqlNode} and {@link RexNode}.</p>
+ * {@link org.apache.calcite.sql.SqlNode} and {@link RexNode}.</p>
  *
  * <p>If this type of relational expression has some particular planner rules,
  * it should implement the <em>public static</em> method
  * {@link AbstractRelNode#register}.</p>
  *
  * <p>When a relational expression comes to be implemented, the system allocates
- * a {@link org.eigenbase.relopt.RelImplementor} to manage the process. Every
+ * a {@link org.apache.calcite.plan.RelImplementor} to manage the process. Every
  * implementable relational expression has a {@link RelTraitSet} describing its
  * physical attributes. The RelTraitSet always contains a {@link Convention}
  * describing how the expression passes data to its consuming
@@ -44,9 +56,10 @@ import org.eigenbase.rex.*;
  * those traits configured by the RelNode itself).</p>
  *
  * <p>For each calling-convention, there is a corresponding sub-interface of
- * RelNode. For example, {@code net.hydromatic.optiq.rules.java.EnumerableRel}
+ * RelNode. For example,
+ * {@code org.apache.calcite.adapter.enumerable.EnumerableRel}
  * has operations to manage the conversion to a graph of
- * {@code net.hydromatic.optiq.rules.java.EnumerableConvention}
+ * {@code org.apache.calcite.adapter.enumerable.EnumerableConvention}
  * calling-convention, and it interacts with a
  * {@code EnumerableRelImplementor}.</p>
  *
@@ -75,8 +88,8 @@ public interface RelNode extends RelOptNode, Cloneable {
   List<RexNode> getChildExps();
 
   /**
-   * Return the CallingConvention trait from this RelNode's {@link
-   * #getTraitSet() trait set}.
+   * Return the CallingConvention trait from this RelNode's
+   * {@link #getTraitSet() trait set}.
    *
    * @return this RelNode's CallingConvention
    */
@@ -122,8 +135,8 @@ public interface RelNode extends RelOptNode, Cloneable {
 
   /**
    * Returns the sub-query this relational expression belongs to. A sub-query
-   * determines the scope for correlating variables (see {@link
-   * #setCorrelVariable(String)}).
+   * determines the scope for correlating variables (see
+   * {@link #setCorrelVariable(String)}).
    *
    * @return Sub-query
    */
@@ -135,8 +148,8 @@ public interface RelNode extends RelOptNode, Cloneable {
   RelDataType getRowType();
 
   /**
-   * Returns the type of the rows expected for an input. Defaults to {@link
-   * #getRowType}.
+   * Returns the type of the rows expected for an input. Defaults to
+   * {@link #getRowType}.
    *
    * @param ordinalInParent input's 0-based ordinal with respect to this
    *                        parent rel
@@ -188,9 +201,9 @@ public interface RelNode extends RelOptNode, Cloneable {
   void collectVariablesSet(Set<String> variableSet);
 
   /**
-   * Interacts with the {@link RelVisitor} in a {@link
-   * org.eigenbase.util.Glossary#VISITOR_PATTERN visitor pattern} to traverse
-   * the tree of relational expressions.
+   * Interacts with the {@link RelVisitor} in a
+   * {@link org.apache.calcite.util.Glossary#VISITOR_PATTERN visitor pattern} to
+   * traverse the tree of relational expressions.
    */
   void childrenAccept(RelVisitor visitor);
 
@@ -218,9 +231,10 @@ public interface RelNode extends RelOptNode, Cloneable {
   /**
    * Describes the inputs and attributes of this relational expression.
    * Each node should call {@code super.explain}, then call the
-   * {@link RelWriterImpl#input(String, RelNode)}
-   * and {@link RelWriterImpl#item(String, Object)} methods for each input
-   * and attribute.
+   * {@link org.apache.calcite.rel.externalize.RelWriterImpl#input(String, RelNode)}
+   * and
+   * {@link org.apache.calcite.rel.externalize.RelWriterImpl#item(String, Object)}
+   * methods for each input and attribute.
    *
    * @param pw Plan writer
    */
@@ -262,7 +276,7 @@ public interface RelNode extends RelOptNode, Cloneable {
   /**
    * Returns the name of this relational expression's class, sans package
    * name, for use in explain. For example, for a <code>
-   * org.eigenbase.rel.ArrayRel.ArrayReader</code>, this method returns
+   * org.apache.calcite.rel.ArrayRel.ArrayReader</code>, this method returns
    * "ArrayReader".
    */
   String getRelTypeName();
@@ -314,9 +328,9 @@ public interface RelNode extends RelOptNode, Cloneable {
    * expression.
    *
    * <p>The planner calls this method this first time that it sees a
-   * relational expression of this class. The derived class should call {@link
-   * org.eigenbase.relopt.RelOptPlanner#addRule} for each rule, and then call
-   * {@code super.register}.</p>
+   * relational expression of this class. The derived class should call
+   * {@link org.apache.calcite.plan.RelOptPlanner#addRule} for each rule, and
+   * then call {@code super.register}.</p>
    */
   void register(RelOptPlanner planner);
 
@@ -324,7 +338,7 @@ public interface RelNode extends RelOptNode, Cloneable {
    * Returns whether the result of this relational expression is uniquely
    * identified by this columns with the given ordinals.
    *
-   * <p>For example, if this relational expression is a TableAccessRel to
+   * <p>For example, if this relational expression is a LogicalTableScan to
    * T(A, B, C, D) whose key is (A, B), then isKey([0, 1]) yields true,
    * and isKey([0]) and isKey([0, 2]) yields false.</p>
    *

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/RelShuttle.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/RelShuttle.java b/core/src/main/java/org/apache/calcite/rel/RelShuttle.java
index c103879..c1e1915 100644
--- a/core/src/main/java/org/apache/calcite/rel/RelShuttle.java
+++ b/core/src/main/java/org/apache/calcite/rel/RelShuttle.java
@@ -14,35 +14,48 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel;
+package org.apache.calcite.rel;
+
+import org.apache.calcite.rel.core.Correlator;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.core.TableFunctionScan;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.rel.logical.LogicalAggregate;
+import org.apache.calcite.rel.logical.LogicalFilter;
+import org.apache.calcite.rel.logical.LogicalIntersect;
+import org.apache.calcite.rel.logical.LogicalJoin;
+import org.apache.calcite.rel.logical.LogicalMinus;
+import org.apache.calcite.rel.logical.LogicalProject;
+import org.apache.calcite.rel.logical.LogicalUnion;
+import org.apache.calcite.rel.logical.LogicalValues;
 
 /**
  * Visitor that has methods for the common logical relational expressions.
  */
 public interface RelShuttle {
-  RelNode visit(TableAccessRelBase scan);
+  RelNode visit(TableScan scan);
 
-  RelNode visit(TableFunctionRelBase scan);
+  RelNode visit(TableFunctionScan scan);
 
-  RelNode visit(ValuesRel values);
+  RelNode visit(LogicalValues values);
 
-  RelNode visit(FilterRel filter);
+  RelNode visit(LogicalFilter filter);
 
-  RelNode visit(ProjectRel project);
+  RelNode visit(LogicalProject project);
 
-  RelNode visit(JoinRel join);
+  RelNode visit(LogicalJoin join);
 
-  RelNode visit(CorrelatorRel correlator);
+  RelNode visit(Correlator correlator);
 
-  RelNode visit(UnionRel union);
+  RelNode visit(LogicalUnion union);
 
-  RelNode visit(IntersectRel intersect);
+  RelNode visit(LogicalIntersect intersect);
 
-  RelNode visit(MinusRel minus);
+  RelNode visit(LogicalMinus minus);
 
-  RelNode visit(AggregateRel aggregate);
+  RelNode visit(LogicalAggregate aggregate);
 
-  RelNode visit(SortRel sort);
+  RelNode visit(Sort sort);
 
   RelNode visit(RelNode other);
 }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/RelShuttleImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/RelShuttleImpl.java b/core/src/main/java/org/apache/calcite/rel/RelShuttleImpl.java
index b3d3502..6b7cb96 100644
--- a/core/src/main/java/org/apache/calcite/rel/RelShuttleImpl.java
+++ b/core/src/main/java/org/apache/calcite/rel/RelShuttleImpl.java
@@ -14,19 +14,30 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel;
+package org.apache.calcite.rel;
+
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.rel.core.Correlator;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.core.TableFunctionScan;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.rel.logical.LogicalAggregate;
+import org.apache.calcite.rel.logical.LogicalFilter;
+import org.apache.calcite.rel.logical.LogicalIntersect;
+import org.apache.calcite.rel.logical.LogicalJoin;
+import org.apache.calcite.rel.logical.LogicalMinus;
+import org.apache.calcite.rel.logical.LogicalProject;
+import org.apache.calcite.rel.logical.LogicalUnion;
+import org.apache.calcite.rel.logical.LogicalValues;
+import org.apache.calcite.util.Stacks;
 
 import java.util.ArrayList;
 import java.util.List;
 
-import org.eigenbase.util.Stacks;
-
-import net.hydromatic.linq4j.Ord;
-
 /**
  * Basic implementation of {@link RelShuttle} that calls
  * {@link RelNode#accept(RelShuttle)} on each child, and
- * {@link RelNode#copy(org.eigenbase.relopt.RelTraitSet, java.util.List)} if
+ * {@link RelNode#copy(org.apache.calcite.plan.RelTraitSet, java.util.List)} if
  * any children change.
  */
 public class RelShuttleImpl implements RelShuttle {
@@ -58,51 +69,51 @@ public class RelShuttleImpl implements RelShuttle {
     return rel;
   }
 
-  public RelNode visit(AggregateRel aggregate) {
-    return visitChild(aggregate, 0, aggregate.getChild());
+  public RelNode visit(LogicalAggregate aggregate) {
+    return visitChild(aggregate, 0, aggregate.getInput());
   }
 
-  public RelNode visit(TableAccessRelBase scan) {
+  public RelNode visit(TableScan scan) {
     return scan;
   }
 
-  public RelNode visit(TableFunctionRelBase scan) {
+  public RelNode visit(TableFunctionScan scan) {
     return visitChildren(scan);
   }
 
-  public RelNode visit(ValuesRel values) {
+  public RelNode visit(LogicalValues values) {
     return values;
   }
 
-  public RelNode visit(FilterRel filter) {
-    return visitChild(filter, 0, filter.getChild());
+  public RelNode visit(LogicalFilter filter) {
+    return visitChild(filter, 0, filter.getInput());
   }
 
-  public RelNode visit(ProjectRel project) {
-    return visitChild(project, 0, project.getChild());
+  public RelNode visit(LogicalProject project) {
+    return visitChild(project, 0, project.getInput());
   }
 
-  public RelNode visit(JoinRel join) {
+  public RelNode visit(LogicalJoin join) {
     return visitChildren(join);
   }
 
-  public RelNode visit(CorrelatorRel correlator) {
+  public RelNode visit(Correlator correlator) {
     return visitChildren(correlator);
   }
 
-  public RelNode visit(UnionRel union) {
+  public RelNode visit(LogicalUnion union) {
     return visitChildren(union);
   }
 
-  public RelNode visit(IntersectRel intersect) {
+  public RelNode visit(LogicalIntersect intersect) {
     return visitChildren(intersect);
   }
 
-  public RelNode visit(MinusRel minus) {
+  public RelNode visit(LogicalMinus minus) {
     return visitChildren(minus);
   }
 
-  public RelNode visit(SortRel sort) {
+  public RelNode visit(Sort sort) {
     return visitChildren(sort);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/RelVisitor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/RelVisitor.java b/core/src/main/java/org/apache/calcite/rel/RelVisitor.java
index 82186db..1a77690 100644
--- a/core/src/main/java/org/apache/calcite/rel/RelVisitor.java
+++ b/core/src/main/java/org/apache/calcite/rel/RelVisitor.java
@@ -14,13 +14,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel;
+package org.apache.calcite.rel;
 
 /**
- * A <code>RelVisitor</code> is a Visitor role in the {@link
- * org.eigenbase.util.Glossary#VISITOR_PATTERN visitor pattern} and visits {@link
- * RelNode} objects as the role of Element. Other components in the pattern:
- * {@link RelNode#childrenAccept(RelVisitor)}.
+ * A <code>RelVisitor</code> is a Visitor role in the
+ * {@link org.apache.calcite.util.Glossary#VISITOR_PATTERN visitor pattern} and
+ * visits {@link RelNode} objects as the role of Element. Other components in
+ * the pattern: {@link RelNode#childrenAccept(RelVisitor)}.
  */
 public abstract class RelVisitor {
   //~ Instance fields --------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/RelWriter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/RelWriter.java b/core/src/main/java/org/apache/calcite/rel/RelWriter.java
index bde35d6..fa63bc2 100644
--- a/core/src/main/java/org/apache/calcite/rel/RelWriter.java
+++ b/core/src/main/java/org/apache/calcite/rel/RelWriter.java
@@ -14,12 +14,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel;
+package org.apache.calcite.rel;
 
-import java.util.List;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.util.Pair;
 
-import org.eigenbase.sql.SqlExplainLevel;
-import org.eigenbase.util.Pair;
+import java.util.List;
 
 /**
  * Callback for an expression to dump itself to.
@@ -32,8 +32,9 @@ public interface RelWriter {
    * Prints an explanation of a node, with a list of (term, value) pairs.
    *
    * <p>The term-value pairs are generally gathered by calling
-   * {@link RelNode#explain(RelWriter)}. Each sub-class of
-   * {@link RelNode} calls {@link #input(String, org.eigenbase.rel.RelNode)}
+   * {@link org.apache.calcite.rel.RelNode#explain(RelWriter)}.
+   * Each sub-class of {@link org.apache.calcite.rel.RelNode}
+   * calls {@link #input(String, org.apache.calcite.rel.RelNode)}
    * and {@link #item(String, Object)} to declare term-value pairs.</p>
    *
    * @param rel       Relational expression

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/SingleRel.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/SingleRel.java b/core/src/main/java/org/apache/calcite/rel/SingleRel.java
index c7d6f53..27e0bf6 100644
--- a/core/src/main/java/org/apache/calcite/rel/SingleRel.java
+++ b/core/src/main/java/org/apache/calcite/rel/SingleRel.java
@@ -14,75 +14,78 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel;
+package org.apache.calcite.rel;
 
-import java.util.Collections;
-import java.util.List;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rel.type.RelDataType;
+
+import com.google.common.collect.ImmutableList;
 
-import org.eigenbase.rel.metadata.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.*;
+import java.util.List;
 
 /**
- * A <code>SingleRel</code> is a base class single-input relational expressions.
+ * Abstract base class for relational expressions with a single input.
+ *
+ * <p>It is not required that single-input relational expressions use this
+ * class as a base class. However, default implementations of methods make life
+ * easier.
  */
 public abstract class SingleRel extends AbstractRelNode {
   //~ Instance fields --------------------------------------------------------
 
-  private RelNode child;
+  private RelNode input;
 
   //~ Constructors -----------------------------------------------------------
 
   /**
    * Creates a <code>SingleRel</code>.
    *
-   * @param cluster {@link RelOptCluster}  this relational expression belongs
-   *                to
-   * @param child   input relational expression
+   * @param cluster Cluster this relational expression belongs to
+   * @param input   Input relational expression
    */
   protected SingleRel(
       RelOptCluster cluster,
       RelTraitSet traits,
-      RelNode child) {
+      RelNode input) {
     super(cluster, traits);
-    this.child = child;
+    this.input = input;
   }
 
   //~ Methods ----------------------------------------------------------------
 
-  public RelNode getChild() {
-    return child;
+  public RelNode getInput() {
+    return input;
   }
 
-  // implement RelNode
-  public List<RelNode> getInputs() {
-    return Collections.singletonList(child);
+  @Override public List<RelNode> getInputs() {
+    return ImmutableList.of(input);
   }
 
-  public double getRows() {
-    // Not necessarily correct, but a better default than Rel's 1.0
-    return RelMetadataQuery.getRowCount(child);
+  @Override public double getRows() {
+    // Not necessarily correct, but a better default than AbstractRelNode's 1.0
+    return RelMetadataQuery.getRowCount(input);
   }
 
-  public void childrenAccept(RelVisitor visitor) {
-    visitor.visit(child, 0, this);
+  @Override public void childrenAccept(RelVisitor visitor) {
+    visitor.visit(input, 0, this);
   }
 
   public RelWriter explainTerms(RelWriter pw) {
     return super.explainTerms(pw)
-        .input("child", getChild());
+        .input("input", getInput());
   }
 
-  // override Rel
-  public void replaceInput(
+  @Override public void replaceInput(
       int ordinalInParent,
       RelNode rel) {
     assert ordinalInParent == 0;
-    this.child = rel;
+    this.input = rel;
   }
 
   protected RelDataType deriveRowType() {
-    return child.getRowType();
+    return input.getRowType();
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/convert/Converter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/convert/Converter.java b/core/src/main/java/org/apache/calcite/rel/convert/Converter.java
index ec078de..b84c30a 100644
--- a/core/src/main/java/org/apache/calcite/rel/convert/Converter.java
+++ b/core/src/main/java/org/apache/calcite/rel/convert/Converter.java
@@ -14,16 +14,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.convert;
+package org.apache.calcite.rel.convert;
 
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
+import org.apache.calcite.plan.RelTraitDef;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
 
 /**
- * A relational expression implements the interface <code>ConverterRel</code> to
- * indicate that it converts a physical attribute, or {@link
- * org.eigenbase.relopt.RelTrait trait}, of a relational expression from one
- * value to another.
+ * A relational expression implements the interface <code>Converter</code> to
+ * indicate that it converts a physical attribute, or
+ * {@link org.apache.calcite.plan.RelTrait trait}, of a relational expression
+ * from one value to another.
  *
  * <p>Sometimes this conversion is expensive; for example, to convert a
  * non-distinct to a distinct object stream, we have to clone every object in
@@ -39,11 +40,11 @@ import org.eigenbase.relopt.*;
  * <p>In principle one could devise converters which change multiple traits
  * simultaneously (say change the sort-order and the physical location of a
  * relational expression). In which case, the method {@link #getInputTraits()}
- * would return a {@link org.eigenbase.relopt.RelTraitSet}. But for simplicity,
- * this class only allows one trait to be converted at a time; all other traits
- * are assumed to be preserved.</p>
+ * would return a {@link org.apache.calcite.plan.RelTraitSet}. But for
+ * simplicity, this class only allows one trait to be converted at a
+ * time; all other traits are assumed to be preserved.</p>
  */
-public interface ConverterRel extends RelNode {
+public interface Converter extends RelNode {
   //~ Methods ----------------------------------------------------------------
 
   /**
@@ -70,7 +71,7 @@ public interface ConverterRel extends RelNode {
    *
    * @return child relational expression
    */
-  RelNode getChild();
+  RelNode getInput();
 }
 
-// End ConverterRel.java
+// End Converter.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 6859cc2..15d1375 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
@@ -14,18 +14,23 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.convert;
+package org.apache.calcite.rel.convert;
 
-import org.eigenbase.rel.*;
-import org.eigenbase.rel.metadata.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.util.*;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTraitDef;
+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 org.eigenbase.rel.convert.ConverterRel}.
+ * Abstract implementation of {@link Converter}.
  */
-public abstract class ConverterRelImpl extends SingleRel
-    implements ConverterRel {
+public abstract class ConverterImpl extends SingleRel
+    implements Converter {
   //~ Instance fields --------------------------------------------------------
 
   protected RelTraitSet inTraits;
@@ -34,14 +39,14 @@ public abstract class ConverterRelImpl extends SingleRel
   //~ Constructors -----------------------------------------------------------
 
   /**
-   * Creates a ConverterRelImpl.
+   * Creates a ConverterImpl.
    *
    * @param cluster  planner's cluster
    * @param traitDef the RelTraitDef this converter converts
    * @param traits   the output traits of this converter
    * @param child    child rel (provides input traits)
    */
-  protected ConverterRelImpl(
+  protected ConverterImpl(
       RelOptCluster cluster,
       RelTraitDef traitDef,
       RelTraitSet traits,
@@ -55,7 +60,7 @@ public abstract class ConverterRelImpl extends SingleRel
 
   // implement RelNode
   public RelOptCost computeSelfCost(RelOptPlanner planner) {
-    double dRows = RelMetadataQuery.getRowCount(getChild());
+    double dRows = RelMetadataQuery.getRowCount(getInput());
     double dCpu = dRows;
     double dIo = 0;
     return planner.getCostFactory().makeCost(dRows, dCpu, dIo);
@@ -68,7 +73,7 @@ public abstract class ConverterRelImpl extends SingleRel
   }
 
   public boolean isDistinct() {
-    return getChild().isDistinct();
+    return getInput().isDistinct();
   }
 
   public RelTraitSet getInputTraits() {
@@ -81,4 +86,4 @@ public abstract class ConverterRelImpl extends SingleRel
 
 }
 
-// End ConverterRelImpl.java
+// End ConverterImpl.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/convert/ConverterRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/convert/ConverterRule.java b/core/src/main/java/org/apache/calcite/rel/convert/ConverterRule.java
index 2724c52..55e243e 100644
--- a/core/src/main/java/org/apache/calcite/rel/convert/ConverterRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/convert/ConverterRule.java
@@ -14,10 +14,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.convert;
+package org.apache.calcite.rel.convert;
 
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptRuleOperand;
+import org.apache.calcite.plan.RelTrait;
+import org.apache.calcite.plan.RelTraitDef;
+import org.apache.calcite.rel.RelNode;
 
 import com.google.common.base.Predicates;
 
@@ -117,9 +122,9 @@ public abstract class ConverterRule extends RelOptRule {
       // Don't apply converters to converters that operate
       // on the same RelTraitDef -- otherwise we get
       // an n^2 effect.
-      if (rel instanceof ConverterRel) {
+      if (rel instanceof Converter) {
         if (((ConverterRule) getRule()).getTraitDef()
-            == ((ConverterRel) rel).getTraitDef()) {
+            == ((Converter) rel).getTraitDef()) {
           return false;
         }
       }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/convert/NoneConverter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/convert/NoneConverter.java b/core/src/main/java/org/apache/calcite/rel/convert/NoneConverter.java
index 102d68d..dc1fa08 100644
--- a/core/src/main/java/org/apache/calcite/rel/convert/NoneConverter.java
+++ b/core/src/main/java/org/apache/calcite/rel/convert/NoneConverter.java
@@ -14,22 +14,26 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.convert;
+package org.apache.calcite.rel.convert;
 
-import java.util.List;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.ConventionTraitDef;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.util.Util;
 
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.util.*;
+import java.util.List;
 
 /**
  * <code>NoneConverter</code> converts a plan from <code>inConvention</code> to
- * {@link org.eigenbase.relopt.Convention#NONE}.
+ * {@link org.apache.calcite.plan.Convention#NONE}.
  */
-public class NoneConverterRel extends ConverterRelImpl {
+public class NoneConverter extends ConverterImpl {
   //~ Constructors -----------------------------------------------------------
 
-  public NoneConverterRel(
+  public NoneConverter(
       RelOptCluster cluster,
       RelNode child) {
     super(
@@ -44,7 +48,7 @@ public class NoneConverterRel extends ConverterRelImpl {
 
   public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
     assert traitSet.comprises(Convention.NONE);
-    return new NoneConverterRel(
+    return new NoneConverter(
         getCluster(),
         sole(inputs));
   }
@@ -55,4 +59,4 @@ public class NoneConverterRel extends ConverterRelImpl {
   }
 }
 
-// End NoneConverterRel.java
+// End NoneConverter.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/convert/TraitMatchingRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/convert/TraitMatchingRule.java b/core/src/main/java/org/apache/calcite/rel/convert/TraitMatchingRule.java
index 12a6efc..63f621c 100644
--- a/core/src/main/java/org/apache/calcite/rel/convert/TraitMatchingRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/convert/TraitMatchingRule.java
@@ -14,15 +14,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.convert;
+package org.apache.calcite.rel.convert;
 
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptRuleOperandChildPolicy;
+import org.apache.calcite.rel.RelNode;
 
 /**
  * TraitMatchingRule adapts a converter rule, restricting it to fire only when
  * its input already matches the expected output trait. This can be used with
- * {@link org.eigenbase.relopt.hep.HepPlanner} in cases where alternate
+ * {@link org.apache.calcite.plan.hep.HepPlanner} in cases where alternate
  * implementations are available and it is desirable to minimize converters.
  */
 public class TraitMatchingRule extends RelOptRule {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/convert/package-info.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/convert/package-info.java b/core/src/main/java/org/apache/calcite/rel/convert/package-info.java
index 5b813a2..cf1032c 100644
--- a/core/src/main/java/org/apache/calcite/rel/convert/package-info.java
+++ b/core/src/main/java/org/apache/calcite/rel/convert/package-info.java
@@ -19,6 +19,6 @@
  * Defines relational expressions and rules for converting between calling
  * conventions.
  */
-package org.eigenbase.rel.convert;
+package org.apache.calcite.rel.convert;
 
 // End package-info.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 6de37b3..920ef46 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
@@ -14,30 +14,58 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel;
-
-import java.util.*;
-
-import org.eigenbase.rel.metadata.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.resource.Resources;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.sql.validate.*;
-import org.eigenbase.util.*;
-
-import net.hydromatic.linq4j.Ord;
-
-import net.hydromatic.optiq.util.BitSets;
+package org.apache.calcite.rel.core;
+
+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;
+import org.apache.calcite.rel.RelWriter;
+import org.apache.calcite.rel.SingleRel;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rel.type.RelDataTypeFieldImpl;
+import org.apache.calcite.runtime.CalciteException;
+import org.apache.calcite.runtime.Resources;
+import org.apache.calcite.sql.SqlAggFunction;
+import org.apache.calcite.sql.SqlOperatorBinding;
+import org.apache.calcite.sql.SqlUtil;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.validate.SqlValidatorException;
+import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.CompositeList;
+import org.apache.calcite.util.IntList;
+import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
 
+import java.util.AbstractList;
+import java.util.BitSet;
+import java.util.List;
+
 /**
- * <code>AggregateRelBase</code> is an abstract base class for implementations
- * of {@link AggregateRel}.
+ * Relational operator that eliminates
+ * duplicates and computes totals.
+ *
+ * <p>It corresponds to the {@code GROUP BY} operator in a SQL query
+ * statement, together with the aggregate functions in the {@code SELECT}
+ * clause.
+ *
+ * <p>Rules:
+ *
+ * <ul>
+ * <li>{@link org.apache.calcite.rel.rules.AggregateProjectPullUpConstantsRule}
+ * <li>{@link org.apache.calcite.rel.rules.AggregateExpandDistinctAggregatesRule}
+ * <li>{@link org.apache.calcite.rel.rules.AggregateReduceFunctionsRule}.
+ * </ul>
  */
-public abstract class AggregateRelBase extends SingleRel {
+public abstract class Aggregate extends SingleRel {
   //~ Instance fields --------------------------------------------------------
 
   protected final List<AggregateCall> aggCalls;
@@ -46,7 +74,7 @@ public abstract class AggregateRelBase extends SingleRel {
   //~ Constructors -----------------------------------------------------------
 
   /**
-   * Creates an AggregateRelBase.
+   * Creates an Aggregate.
    *
    * @param cluster  Cluster
    * @param traits   Traits
@@ -54,7 +82,7 @@ public abstract class AggregateRelBase extends SingleRel {
    * @param groupSet Bit set of grouping fields
    * @param aggCalls Collection of calls to aggregate functions
    */
-  protected AggregateRelBase(
+  protected Aggregate(
       RelOptCluster cluster,
       RelTraitSet traits,
       RelNode child,
@@ -75,9 +103,9 @@ public abstract class AggregateRelBase extends SingleRel {
   }
 
   /**
-   * Creates an AggregateRelBase by parsing serialized output.
+   * Creates an Aggregate by parsing serialized output.
    */
-  protected AggregateRelBase(RelInput input) {
+  protected Aggregate(RelInput input) {
     this(input.getCluster(), input.getTraitSet(), input.getInput(),
         input.getBitSet("group"), input.getAggregateCalls("aggs"));
   }
@@ -91,9 +119,9 @@ public abstract class AggregateRelBase extends SingleRel {
 
   /** Creates a copy of this aggregate.
    *
-   * @see #copy(org.eigenbase.relopt.RelTraitSet, java.util.List)
+   * @see #copy(org.apache.calcite.plan.RelTraitSet, java.util.List)
    */
-  public abstract AggregateRelBase copy(RelTraitSet traitSet, RelNode input,
+  public abstract Aggregate copy(RelTraitSet traitSet, RelNode input,
       BitSet groupSet, List<AggregateCall> aggCalls);
 
   // implement RelNode
@@ -166,7 +194,7 @@ public abstract class AggregateRelBase extends SingleRel {
 
   public RelOptCost computeSelfCost(RelOptPlanner planner) {
     // REVIEW jvs 24-Aug-2008:  This is bogus, but no more bogus
-    // than what's currently in JoinRelBase.
+    // than what's currently in Join.
     double rowCount = RelMetadataQuery.getRowCount(this);
     return planner.getCostFactory().makeCost(rowCount, 0, 0);
   }
@@ -174,12 +202,12 @@ public abstract class AggregateRelBase extends SingleRel {
   protected RelDataType deriveRowType() {
     return deriveRowType(
         getCluster().getTypeFactory(),
-        getChild().getRowType(),
+        getInput().getRowType(),
         groupSet,
         aggCalls);
   }
 
-  /** Computes the row type of an {@code AggregateRelBase} before it exists. */
+  /** Computes the row type of an {@code Aggregate} before it exists. */
   public static RelDataType deriveRowType(RelDataTypeFactory typeFactory,
       final RelDataType inputRowType, BitSet groupSet,
       final List<AggregateCall> aggCalls) {
@@ -256,9 +284,9 @@ public abstract class AggregateRelBase extends SingleRel {
   //~ Inner Classes ----------------------------------------------------------
 
   /**
-   * Implementation of the {@link SqlOperatorBinding} interface for an {@link
-   * AggregateCall aggregate call} applied to a set of operands in the context
-   * of a {@link AggregateRel}.
+   * Implementation of the {@link SqlOperatorBinding} interface for an
+   * {@link AggregateCall aggregate call} applied to a set of operands in the
+   * context of a {@link org.apache.calcite.rel.logical.LogicalAggregate}.
    */
   public static class AggCallBinding extends SqlOperatorBinding {
     private final List<RelDataType> operands;
@@ -268,7 +296,7 @@ public abstract class AggregateRelBase extends SingleRel {
      * Creates an AggCallBinding
      *
      * @param typeFactory  Type factory
-     * @param aggFunction  Aggregation function
+     * @param aggFunction  Aggregate function
      * @param operands     Data types of operands
      * @param groupCount   Number of columns in the GROUP BY clause
      */
@@ -284,7 +312,7 @@ public abstract class AggregateRelBase extends SingleRel {
           : "operands of aggregate call should not be null";
       assert groupCount >= 0
           : "number of group by columns should be greater than zero in "
-            + "aggregate call. Got " + groupCount;
+          + "aggregate call. Got " + groupCount;
     }
 
     @Override public int getGroupCount() {
@@ -299,11 +327,11 @@ public abstract class AggregateRelBase extends SingleRel {
       return operands.get(ordinal);
     }
 
-    public EigenbaseException newError(
+    public CalciteException newError(
         Resources.ExInst<SqlValidatorException> e) {
       return SqlUtil.newContextException(SqlParserPos.ZERO, e);
     }
   }
 }
 
-// End AggregateRelBase.java
+// End Aggregate.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/core/AggregateCall.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/AggregateCall.java b/core/src/main/java/org/apache/calcite/rel/core/AggregateCall.java
index 651c2f3..fa1620c 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/AggregateCall.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/AggregateCall.java
@@ -14,24 +14,27 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel;
+package org.apache.calcite.rel.core;
 
-import java.util.*;
-
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.type.SqlTypeUtil;
-import org.eigenbase.util.mapping.Mappings;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.sql.SqlAggFunction;
+import org.apache.calcite.sql.type.SqlTypeUtil;
+import org.apache.calcite.util.mapping.Mappings;
 
 import com.google.common.collect.ImmutableList;
 
+import java.util.List;
+
 /**
- * Call to an aggregation function within an {@link AggregateRel}.
+ * Call to an aggFunction function within an
+ * {@link org.apache.calcite.rel.logical.LogicalAggregate}.
  */
 public class AggregateCall {
   //~ Instance fields --------------------------------------------------------
 
-  private final Aggregation aggregation;
+  private final SqlAggFunction aggFunction;
 
   private final boolean distinct;
   public final RelDataType type;
@@ -46,24 +49,24 @@ public class AggregateCall {
   /**
    * Creates an AggregateCall.
    *
-   * @param aggregation Aggregation
+   * @param aggFunction Aggregate function
    * @param distinct    Whether distinct
    * @param argList     List of ordinals of arguments
    * @param type        Result type
    * @param name        Name (may be null)
    */
   public AggregateCall(
-      Aggregation aggregation,
+      SqlAggFunction aggFunction,
       boolean distinct,
       List<Integer> argList,
       RelDataType type,
       String name) {
     this.type = type;
     this.name = name;
-    assert aggregation != null;
+    assert aggFunction != null;
     assert argList != null;
     assert type != null;
-    this.aggregation = aggregation;
+    this.aggFunction = aggFunction;
 
     this.argList = ImmutableList.copyOf(argList);
     this.distinct = distinct;
@@ -80,8 +83,8 @@ public class AggregateCall {
           input.getCluster().getTypeFactory();
       final List<RelDataType> types =
           SqlTypeUtil.projectTypes(input.getRowType(), argList);
-      final AggregateRelBase.AggCallBinding callBinding =
-          new AggregateRelBase.AggCallBinding(typeFactory, aggFunction, types,
+      final Aggregate.AggCallBinding callBinding =
+          new Aggregate.AggCallBinding(typeFactory, aggFunction, types,
               groupCount);
       type = aggFunction.inferReturnType(callBinding);
     }
@@ -99,12 +102,12 @@ public class AggregateCall {
   }
 
   /**
-   * Returns the Aggregation.
+   * Returns the aggregate function.
    *
-   * @return aggregation
+   * @return aggregate function
    */
-  public final Aggregation getAggregation() {
-    return aggregation;
+  public final SqlAggFunction getAggregation() {
+    return aggFunction;
   }
 
   /**
@@ -143,11 +146,11 @@ public class AggregateCall {
    */
   public AggregateCall rename(String name) {
     // no need to copy argList - already immutable
-    return new AggregateCall(aggregation, distinct, argList, type, name);
+    return new AggregateCall(aggFunction, distinct, argList, type, name);
   }
 
   public String toString() {
-    StringBuilder buf = new StringBuilder(aggregation.getName());
+    StringBuilder buf = new StringBuilder(aggFunction.getName());
     buf.append("(");
     if (distinct) {
       buf.append((argList.size() == 0) ? "DISTINCT" : "DISTINCT ");
@@ -170,27 +173,28 @@ public class AggregateCall {
       return false;
     }
     AggregateCall other = (AggregateCall) o;
-    return aggregation.equals(other.aggregation)
+    return aggFunction.equals(other.aggFunction)
         && (distinct == other.distinct)
         && argList.equals(other.argList);
   }
 
   // override Object
   public int hashCode() {
-    return aggregation.hashCode() + argList.hashCode();
+    return aggFunction.hashCode() + argList.hashCode();
   }
 
   /**
-   * Creates a binding of this call in the context of an {@link AggregateRel},
+   * Creates a binding of this call in the context of an
+   * {@link org.apache.calcite.rel.logical.LogicalAggregate},
    * which can then be used to infer the return type.
    */
-  public AggregateRelBase.AggCallBinding createBinding(
-      AggregateRelBase aggregateRelBase) {
-    final RelDataType rowType = aggregateRelBase.getChild().getRowType();
+  public Aggregate.AggCallBinding createBinding(
+      Aggregate aggregateRelBase) {
+    final RelDataType rowType = aggregateRelBase.getInput().getRowType();
 
-    return new AggregateRelBase.AggCallBinding(
+    return new Aggregate.AggCallBinding(
         aggregateRelBase.getCluster().getTypeFactory(),
-        (SqlAggFunction) aggregation,
+        (SqlAggFunction) aggFunction,
         SqlTypeUtil.projectTypes(rowType, argList),
         aggregateRelBase.getGroupCount());
   }
@@ -202,22 +206,22 @@ public class AggregateCall {
    * @return AggregateCall that suits new inputs and GROUP BY columns
    */
   public AggregateCall copy(List<Integer> args) {
-    return new AggregateCall(aggregation, distinct, args, type, name);
+    return new AggregateCall(aggFunction, distinct, args, type, name);
   }
 
   /**
    * Creates equivalent AggregateCall that is adapted to a new input types
    * and/or number of columns in GROUP BY.
    *
-   * @param input relation that will be used as a child of AggregateRel
+   * @param input relation that will be used as a child of aggregate
    * @param aggArgs argument indices of the new call in the input
-   * @param oldGroupKeyCount number of columns in GROUP BY of old AggregateRel
-   * @param newGroupKeyCount number of columns in GROUP BY of new AggregateRel
+   * @param oldGroupKeyCount number of columns in GROUP BY of old aggregate
+   * @param newGroupKeyCount number of columns in GROUP BY of new aggregate
    * @return AggregateCall that suits new inputs and GROUP BY columns
    */
   public AggregateCall adaptTo(RelNode input, List<Integer> aggArgs,
       int oldGroupKeyCount, int newGroupKeyCount) {
-    final SqlAggFunction sqlAgg = (SqlAggFunction) aggregation;
+    final SqlAggFunction sqlAgg = (SqlAggFunction) aggFunction;
     // The return type of aggregate call need to be recomputed.
     // Since it might depend on the number of columns in GROUP BY.
     final RelDataType newType =

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 cfa1e3d..803c783 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
@@ -14,22 +14,32 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel;
-
-import java.util.*;
-
-import org.eigenbase.rel.metadata.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.rex.*;
+package org.apache.calcite.rel.core;
+
+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.RelCollation;
+import org.apache.calcite.rel.RelCollationImpl;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelWriter;
+import org.apache.calcite.rel.SingleRel;
+import org.apache.calcite.rel.logical.LogicalFilter;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexProgram;
 
 import com.google.common.collect.ImmutableList;
 
+import java.util.List;
+
 /**
- * <code>CalcRelBase</code> is an abstract base class for implementations of
- * {@link CalcRel}.
+ * <code>Calc</code> is an abstract base class for implementations of
+ * {@link org.apache.calcite.rel.logical.LogicalCalc}.
  */
-public abstract class CalcRelBase extends SingleRel {
+public abstract class Calc extends SingleRel {
   //~ Instance fields --------------------------------------------------------
 
   protected final RexProgram program;
@@ -37,7 +47,7 @@ public abstract class CalcRelBase extends SingleRel {
 
   //~ Constructors -----------------------------------------------------------
 
-  protected CalcRelBase(
+  protected Calc(
       RelOptCluster cluster,
       RelTraitSet traits,
       RelNode child,
@@ -53,15 +63,12 @@ public abstract class CalcRelBase extends SingleRel {
 
   //~ Methods ----------------------------------------------------------------
 
-  @Override
-  public final CalcRelBase copy(RelTraitSet traitSet, List<RelNode> inputs) {
+  @Override public final Calc copy(RelTraitSet traitSet, List<RelNode> inputs) {
     return copy(traitSet, sole(inputs), program, collationList);
   }
 
-  /**
-   * Creates a copy of this {@code CalcRelBase}.
-   */
-  public abstract CalcRelBase copy(
+  /** Creates a copy of this {@code Calc}. */
+  public abstract Calc copy(
       RelTraitSet traitSet,
       RelNode child,
       RexProgram program,
@@ -72,7 +79,7 @@ public abstract class CalcRelBase extends SingleRel {
         "program's input type",
         program.getInputRowType(),
         "child's output type",
-        getChild().getRowType(),
+        getInput().getRowType(),
         fail)) {
       return false;
     }
@@ -104,8 +111,8 @@ public abstract class CalcRelBase extends SingleRel {
   }
 
   public double getRows() {
-    return FilterRel.estimateFilteredRows(
-        getChild(),
+    return LogicalFilter.estimateFilteredRows(
+        getInput(),
         program);
   }
 
@@ -116,7 +123,7 @@ public abstract class CalcRelBase extends SingleRel {
   public RelOptCost computeSelfCost(RelOptPlanner planner) {
     double dRows = RelMetadataQuery.getRowCount(this);
     double dCpu =
-        RelMetadataQuery.getRowCount(getChild())
+        RelMetadataQuery.getRowCount(getInput())
             * program.getExprCount();
     double dIo = 0;
     return planner.getCostFactory().makeCost(dRows, dCpu, dIo);
@@ -127,4 +134,4 @@ public abstract class CalcRelBase extends SingleRel {
   }
 }
 
-// End CalcRelBase.java
+// End Calc.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/core/Collect.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/Collect.java b/core/src/main/java/org/apache/calcite/rel/core/Collect.java
index 0dc9f80..522672a 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/Collect.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/Collect.java
@@ -14,13 +14,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel;
+package org.apache.calcite.rel.core;
 
-import java.util.List;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelInput;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelWriter;
+import org.apache.calcite.rel.SingleRel;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.sql.type.SqlTypeUtil;
 
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.type.*;
+import java.util.List;
 
 /**
  * A relational expression that collapses multiple rows into one.
@@ -29,12 +36,12 @@ import org.eigenbase.sql.type.*;
  *
  * <ul>
  * <li>{@code net.sf.farrago.fennel.rel.FarragoMultisetSplitterRule}
- * creates a CollectRel from a call to
- * {@link org.eigenbase.sql.fun.SqlMultisetValueConstructor} or to
- * {@link org.eigenbase.sql.fun.SqlMultisetQueryConstructor}.</li>
+ * creates a Collect from a call to
+ * {@link org.apache.calcite.sql.fun.SqlMultisetValueConstructor} or to
+ * {@link org.apache.calcite.sql.fun.SqlMultisetQueryConstructor}.</li>
  * </ul>
  */
-public class CollectRel extends SingleRel {
+public class Collect extends SingleRel {
   //~ Instance fields --------------------------------------------------------
 
   protected final String fieldName;
@@ -42,13 +49,13 @@ public class CollectRel extends SingleRel {
   //~ Constructors -----------------------------------------------------------
 
   /**
-   * Creates a CollectRel.
+   * Creates a Collect.
    *
    * @param cluster   Cluster
    * @param child     Child relational expression
    * @param fieldName Name of the sole output field
    */
-  public CollectRel(
+  public Collect(
       RelOptCluster cluster,
       RelTraitSet traitSet,
       RelNode child,
@@ -58,27 +65,26 @@ public class CollectRel extends SingleRel {
   }
 
   /**
-   * Creates a CollectRel by parsing serialized output.
+   * Creates a Collect by parsing serialized output.
    */
-  public CollectRel(RelInput input) {
+  public Collect(RelInput input) {
     this(input.getCluster(), input.getTraitSet(), input.getInput(),
         input.getString("field"));
   }
 
   //~ Methods ----------------------------------------------------------------
 
-  @Override
-  public final RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+  @Override public final RelNode copy(RelTraitSet traitSet,
+      List<RelNode> inputs) {
     return copy(traitSet, sole(inputs));
   }
 
   public RelNode copy(RelTraitSet traitSet, RelNode input) {
     assert traitSet.containsIfApplicable(Convention.NONE);
-    return new CollectRel(getCluster(), traitSet, input, fieldName);
+    return new Collect(getCluster(), traitSet, input, fieldName);
   }
 
-  @Override
-  public RelWriter explainTerms(RelWriter pw) {
+  @Override public RelWriter explainTerms(RelWriter pw) {
     return super.explainTerms(pw)
         .item("field", fieldName);
   }
@@ -92,8 +98,7 @@ public class CollectRel extends SingleRel {
     return fieldName;
   }
 
-  @Override
-  protected RelDataType deriveRowType() {
+  @Override protected RelDataType deriveRowType() {
     return deriveCollectRowType(this, fieldName);
   }
 
@@ -107,7 +112,7 @@ public class CollectRel extends SingleRel {
   public static RelDataType deriveCollectRowType(
       SingleRel rel,
       String fieldName) {
-    RelDataType childType = rel.getChild().getRowType();
+    RelDataType childType = rel.getInput().getRowType();
     assert childType.isStruct();
     final RelDataTypeFactory typeFactory = rel.getCluster().getTypeFactory();
     RelDataType ret =
@@ -120,4 +125,4 @@ public class CollectRel extends SingleRel {
   }
 }
 
-// End CollectRel.java
+// End Collect.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/core/Correlation.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/Correlation.java b/core/src/main/java/org/apache/calcite/rel/core/Correlation.java
index cd0de34..7aed265 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/Correlation.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/Correlation.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel;
+package org.apache.calcite.rel.core;
 
 /**
  * Describes the necessary parameters for an implementation in order to

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/core/Correlator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/Correlator.java b/core/src/main/java/org/apache/calcite/rel/core/Correlator.java
index eff6c38..a80d324 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/Correlator.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/Correlator.java
@@ -14,25 +14,37 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel;
+package org.apache.calcite.rel.core;
 
-import java.util.*;
-
-import org.eigenbase.relopt.*;
-import org.eigenbase.rex.*;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelInput;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelShuttle;
+import org.apache.calcite.rel.RelWriter;
+import org.apache.calcite.rex.RexNode;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
 
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
 /**
- * A <code>CorrelatorRel</code> behaves like a kind of {@link JoinRel}, but
- * works by setting variables in its environment and restarting its right-hand
- * input.
+ * A relational operator that performs nested-loop joins.
+ *
+ * <p>It behaves like a kind of {@link Join},
+ * but works by setting variables in its environment and restarting its
+ * right-hand input.
  *
- * <p>A CorrelatorRel is used to represent a correlated query. One
+ * <p>A Correlator is used to represent a correlated query. One
  * implementation strategy is to de-correlate the expression.
+ *
+ * @see Correlation
  */
-public final class CorrelatorRel extends JoinRelBase {
+public final class Correlator extends Join {
   //~ Instance fields --------------------------------------------------------
 
   protected final ImmutableList<Correlation> correlations;
@@ -40,7 +52,7 @@ public final class CorrelatorRel extends JoinRelBase {
   //~ Constructors -----------------------------------------------------------
 
   /**
-   * Creates a CorrelatorRel.
+   * Creates a Correlator.
    *
    * @param cluster      cluster this relational expression belongs to
    * @param left         left input relational expression
@@ -50,7 +62,7 @@ public final class CorrelatorRel extends JoinRelBase {
    *                     row arrives from the left input
    * @param joinType     join type
    */
-  public CorrelatorRel(
+  public Correlator(
       RelOptCluster cluster,
       RelNode left,
       RelNode right,
@@ -71,16 +83,16 @@ public final class CorrelatorRel extends JoinRelBase {
   }
 
   /**
-   * Creates a CorrelatorRel with no join condition.
+   * Creates a Correlator with no join condition.
    *
-   * @param cluster      cluster this relational expression belongs to
+   * @param cluster      Cluster that this relational expression belongs to
    * @param left         left input relational expression
    * @param right        right input relational expression
    * @param correlations set of expressions to set as variables each time a
    *                     row arrives from the left input
    * @param joinType     join type
    */
-  public CorrelatorRel(
+  public Correlator(
       RelOptCluster cluster,
       RelNode left,
       RelNode right,
@@ -96,9 +108,9 @@ public final class CorrelatorRel extends JoinRelBase {
   }
 
   /**
-   * Creates a CorrelatorRel by parsing serialized output.
+   * Creates a Correlator by parsing serialized output.
    */
-  public CorrelatorRel(RelInput input) {
+  public Correlator(RelInput input) {
     this(
         input.getCluster(), input.getInputs().get(0),
         input.getInputs().get(1), getCorrelations(input),
@@ -121,11 +133,10 @@ public final class CorrelatorRel extends JoinRelBase {
 
   //~ Methods ----------------------------------------------------------------
 
-  @Override
-  public CorrelatorRel copy(RelTraitSet traitSet, RexNode conditionExpr,
+  @Override public Correlator copy(RelTraitSet traitSet, RexNode conditionExpr,
       RelNode left, RelNode right, JoinRelType joinType, boolean semiJoinDone) {
     assert traitSet.containsIfApplicable(Convention.NONE);
-    return new CorrelatorRel(
+    return new Correlator(
         getCluster(),
         left,
         right,
@@ -152,4 +163,4 @@ public final class CorrelatorRel extends JoinRelBase {
   }
 }
 
-// End CorrelatorRel.java
+// End Correlator.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/core/Empty.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/Empty.java b/core/src/main/java/org/apache/calcite/rel/core/Empty.java
index cb14057..772b946 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/Empty.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/Empty.java
@@ -14,46 +14,50 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel;
+package org.apache.calcite.rel.core;
 
-import java.util.List;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.AbstractRelNode;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelWriter;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlExplainLevel;
 
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
+import java.util.List;
 
 /**
- * <code>EmptyRel</code> represents a relational expression with zero rows.
+ * Relational expression with zero rows.
  *
- * <p>EmptyRel can not be implemented, but serves as a token for rules to match
+ * <p>Empty can not be implemented, but serves as a token for rules to match
  * so that empty sections of queries can be eliminated.
  *
  * <p>Rules:
  *
  * <ul>
  * <li>Created by {@code net.sf.farrago.query.FarragoReduceValuesRule}</li>
- * <li>Triggers {@link org.eigenbase.rel.rules.RemoveEmptyRules}</li>
+ * <li>Triggers {@link org.apache.calcite.rel.rules.EmptyPruneRules}</li>
  * </ul>
  *
- * @see org.eigenbase.rel.ValuesRel
+ * @see org.apache.calcite.rel.logical.LogicalValues
+ * @see OneRow
  */
-public class EmptyRel extends AbstractRelNode {
+public class Empty extends AbstractRelNode {
   //~ Constructors -----------------------------------------------------------
 
   /**
-   * Creates a new EmptyRel.
+   * Creates a new Empty.
    *
    * @param cluster Cluster
    * @param rowType row type for tuples which would be produced by this rel if
    *                it actually produced any, but it doesn't (see, philosophy is
    *                good for something after all!)
    */
-  public EmptyRel(
-      RelOptCluster cluster,
-      RelDataType rowType) {
-    super(
-        cluster,
-        cluster.traitSetOf(Convention.NONE));
+  public Empty(RelOptCluster cluster, RelDataType rowType) {
+    super(cluster, cluster.traitSetOf(Convention.NONE));
     this.rowType = rowType;
   }
 
@@ -66,23 +70,19 @@ public class EmptyRel extends AbstractRelNode {
     return this;
   }
 
-  // implement RelNode
-  protected RelDataType deriveRowType() {
+  @Override protected RelDataType deriveRowType() {
     return rowType;
   }
 
-  // implement RelNode
-  public RelOptCost computeSelfCost(RelOptPlanner planner) {
+  @Override public RelOptCost computeSelfCost(RelOptPlanner planner) {
     return planner.getCostFactory().makeZeroCost();
   }
 
-  // implement RelNode
-  public double getRows() {
+  @Override public double getRows() {
     return 0.0;
   }
 
-  // implement RelNode
-  public RelWriter explainTerms(RelWriter pw) {
+  @Override public RelWriter explainTerms(RelWriter pw) {
     return super.explainTerms(pw)
         // For rel digest, include the row type to discriminate
         // this from other empties with different row types.
@@ -94,4 +94,4 @@ public class EmptyRel extends AbstractRelNode {
   }
 }
 
-// End EmptyRel.java
+// End Empty.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 7468a7e..8fbdb0e 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
@@ -14,21 +14,38 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel;
-
-import java.util.List;
-
-import org.eigenbase.rel.metadata.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.rex.*;
+package org.apache.calcite.rel.core;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelInput;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelWriter;
+import org.apache.calcite.rel.SingleRel;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rex.RexChecker;
+import org.apache.calcite.rex.RexLocalRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexUtil;
 
 import com.google.common.collect.ImmutableList;
 
+import java.util.List;
+
 /**
- * <code>FilterRelBase</code> is an abstract base class for implementations of
- * {@link FilterRel}.
+ * Relational expression that iterates over its input
+ * and returns elements for which <code>condition</code> evaluates to
+ * <code>true</code>.
+ *
+ * <p>If the condition allows nulls, then a null value is treated the same as
+ * false.</p>
+ *
+ * @see org.apache.calcite.rel.logical.LogicalFilter
  */
-public abstract class FilterRelBase extends SingleRel {
+public abstract class Filter extends SingleRel {
   //~ Instance fields --------------------------------------------------------
 
   protected final RexNode condition;
@@ -38,14 +55,13 @@ public abstract class FilterRelBase extends SingleRel {
   /**
    * Creates a filter.
    *
-   * @param cluster   {@link RelOptCluster}  this relational expression belongs
-   *                  to
+   * @param cluster   Cluster that this relational expression belongs to
    * @param traits    the traits of this rel
    * @param child     input relational expression
    * @param condition boolean expression which determines whether a row is
    *                  allowed to pass
    */
-  protected FilterRelBase(
+  protected Filter(
       RelOptCluster cluster,
       RelTraitSet traits,
       RelNode child,
@@ -59,25 +75,24 @@ public abstract class FilterRelBase extends SingleRel {
   }
 
   /**
-   * Creates a FilterRelBase by parsing serialized output.
+   * Creates a Filter by parsing serialized output.
    */
-  protected FilterRelBase(RelInput input) {
+  protected Filter(RelInput input) {
     this(input.getCluster(), input.getTraitSet(), input.getInput(),
         input.getExpression("condition"));
   }
 
   //~ Methods ----------------------------------------------------------------
 
-  @Override
-  public final RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+  @Override public final RelNode copy(RelTraitSet traitSet,
+      List<RelNode> inputs) {
     return copy(traitSet, sole(inputs), getCondition());
   }
 
-  public abstract FilterRelBase copy(RelTraitSet traitSet, RelNode input,
+  public abstract Filter copy(RelTraitSet traitSet, RelNode input,
       RexNode condition);
 
-  @Override
-  public List<RexNode> getChildExps() {
+  @Override public List<RexNode> getChildExps() {
     return ImmutableList.of(condition);
   }
 
@@ -86,7 +101,7 @@ public abstract class FilterRelBase extends SingleRel {
   }
 
   @Override public boolean isValid(boolean fail) {
-    final RexChecker checker = new RexChecker(getChild().getRowType(), fail);
+    final RexChecker checker = new RexChecker(getInput().getRowType(), fail);
     condition.accept(checker);
     if (checker.getFailureCount() > 0) {
       assert !fail;
@@ -97,7 +112,7 @@ public abstract class FilterRelBase extends SingleRel {
 
   public RelOptCost computeSelfCost(RelOptPlanner planner) {
     double dRows = RelMetadataQuery.getRowCount(this);
-    double dCpu = RelMetadataQuery.getRowCount(getChild());
+    double dCpu = RelMetadataQuery.getRowCount(getInput());
     double dIo = 0;
     return planner.getCostFactory().makeCost(dRows, dCpu, dIo);
   }
@@ -105,7 +120,7 @@ public abstract class FilterRelBase extends SingleRel {
   // override RelNode
   public double getRows() {
     return estimateFilteredRows(
-        getChild(),
+        getInput(),
         condition);
   }
 
@@ -134,4 +149,4 @@ public abstract class FilterRelBase extends SingleRel {
   }
 }
 
-// End FilterRelBase.java
+// End Filter.java


[06/58] [abbrv] [partial] incubator-calcite git commit: [CALCITE-306] Standardize code style for "import package.*; "

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
----------------------------------------------------------------------
diff --git a/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml b/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
index 435360a..1ada2df 100644
--- a/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
+++ b/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
@@ -19,8 +19,8 @@ limitations under the License.
     <TestCase name="testCase">
         <Resource name="plan">
             <![CDATA[
-ProjectRel(EXPR$0=[CASE(=('a', 'a'), 1, null)])
-  OneRowRel
+LogicalProject(EXPR$0=[CASE(=('a', 'a'), 1, null)])
+  LogicalOneRow
 ]]>
         </Resource>
         <Resource name="sql">
@@ -37,9 +37,9 @@ ProjectRel(EXPR$0=[$0])
     <TestCase name="testGroup">
         <Resource name="plan">
             <![CDATA[
-AggregateRel(group=[{0}])
-  ProjectRel(DEPTNO=[$7])
-    TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalAggregate(group=[{0}])
+  LogicalProject(DEPTNO=[$7])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
         <Resource name="sql">
@@ -57,9 +57,9 @@ ProjectRel(DEPTNO=[$0])
     <TestCase name="testGroupJustOneAgg">
         <Resource name="plan">
             <![CDATA[
-AggregateRel(group=[{0}], SUM_SAL=[SUM($1)])
-  ProjectRel(DEPTNO=[$7], SAL=[$5])
-    TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalAggregate(group=[{0}], SUM_SAL=[SUM($1)])
+  LogicalProject(DEPTNO=[$7], SAL=[$5])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
         <Resource name="sql">
@@ -77,10 +77,10 @@ ProjectRel(DEPTNO=[$0], EXPR$1=[$1])
     <TestCase name="testGroupExpressionsInsideAndOut">
         <Resource name="plan">
             <![CDATA[
-ProjectRel(EXPR$0=[+($0, 4)], EXPR$1=[$1], EXPR$2=[$2], EXPR$3=[*(2, $3)])
-  AggregateRel(group=[{0}], EXPR$1=[SUM($1)], EXPR$2=[SUM($2)], agg#2=[COUNT()])
-    ProjectRel(DEPTNO=[$7], SAL=[$5], $f2=[+(3, $5)])
-      TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(EXPR$0=[+($0, 4)], EXPR$1=[$1], EXPR$2=[$2], EXPR$3=[*(2, $3)])
+  LogicalAggregate(group=[{0}], EXPR$1=[SUM($1)], EXPR$2=[SUM($2)], agg#2=[COUNT()])
+    LogicalProject(DEPTNO=[$7], SAL=[$5], $f2=[+(3, $5)])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
         <Resource name="sql">
@@ -98,11 +98,11 @@ ProjectRel(EXPR$0=[+($0, 4)], EXPR$1=[$1], EXPR$2=[$2], EXPR$3=[*(2, $3)])
     <TestCase name="testHaving">
         <Resource name="plan">
             <![CDATA[
-ProjectRel(EXPR$0=[$0])
-  FilterRel(condition=[>($1, 10)])
-    AggregateRel(group=[{}], EXPR$0=[SUM($0)], agg#1=[SUM($1)])
-      ProjectRel($f0=[+($5, $5)], SAL=[$5])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(EXPR$0=[$0])
+  LogicalFilter(condition=[>($1, 10)])
+    LogicalAggregate(group=[{}], EXPR$0=[SUM($0)], agg#1=[SUM($1)])
+      LogicalProject($f0=[+($5, $5)], SAL=[$5])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
         <Resource name="sql">
@@ -121,10 +121,10 @@ ProjectRel(EXPR$0=[$0])
     <TestCase name="testGroupBug281">
         <Resource name="plan">
             <![CDATA[
-ProjectRel(NAME=[$0])
-  AggregateRel(group=[{0}])
-    ProjectRel(NAME=[$1])
-      TableAccessRel(table=[[CATALOG, SALES, DEPT]])
+LogicalProject(NAME=[$0])
+  LogicalAggregate(group=[{0}])
+    LogicalProject(NAME=[$1])
+      LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
         </Resource>
         <Resource name="sql">
@@ -143,11 +143,11 @@ ProjectRel(NAME=[$0])
     <TestCase name="testGroupBug281b">
         <Resource name="plan">
             <![CDATA[
-ProjectRel(NAME=[$1], FOO=[$2])
-  ProjectRel(DEPTNO=[$1], NAME=[$0], FOO=[$2])
-    AggregateRel(group=[{0, 1}], FOO=[COUNT()])
-      ProjectRel(NAME=[$1], DEPTNO=[$0])
-        TableAccessRel(table=[[CATALOG, SALES, DEPT]])
+LogicalProject(NAME=[$1], FOO=[$2])
+  LogicalProject(DEPTNO=[$1], NAME=[$0], FOO=[$2])
+    LogicalAggregate(group=[{0, 1}], FOO=[COUNT()])
+      LogicalProject(NAME=[$1], DEPTNO=[$0])
+        LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
         </Resource>
         <Resource name="sql">
@@ -166,9 +166,9 @@ ProjectRel(NAME=[$1], FOO=[$2])
     <TestCase name="testAggDistinct">
         <Resource name="plan">
             <![CDATA[
-AggregateRel(group=[{0}], EXPR$1=[SUM($1)], EXPR$2=[SUM(DISTINCT $1)], EXPR$3=[COUNT()])
-  ProjectRel(DEPTNO=[$7], SAL=[$5])
-    TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalAggregate(group=[{0}], EXPR$1=[SUM($1)], EXPR$2=[SUM(DISTINCT $1)], EXPR$3=[COUNT()])
+  LogicalProject(DEPTNO=[$7], SAL=[$5])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
         <Resource name="sql">
@@ -186,11 +186,11 @@ ProjectRel(DEPTNO=[$0], EXPR$1=[$1], EXPR$2=[$2], EXPR$3=[$3])
     <TestCase name="testUnnest">
         <Resource name="plan">
             <![CDATA[
-ProjectRel(EXPR$0=[$0])
-  UncollectRel
-    ProjectRel(EXPR$0=[$SLICE($0)])
-      CollectRel(field=[EXPR$0])
-        ValuesRel(tuples=[[{ 1 }, { 2 }]])
+LogicalProject(EXPR$0=[$0])
+  Uncollect
+    LogicalProject(EXPR$0=[$SLICE($0)])
+      Collect(field=[EXPR$0])
+        LogicalValues(tuples=[[{ 1 }, { 2 }]])
 ]]>
         </Resource>
         <Resource name="sql">
@@ -213,11 +213,11 @@ ProjectRel(EXPR$0=[$0])
     <TestCase name="testUnnestSubquery">
         <Resource name="plan">
             <![CDATA[
-ProjectRel(DEPTNO=[$0], NAME=[$1])
-  UncollectRel
-    CollectRel(field=[EXPR$0])
-      ProjectRel(DEPTNO=[$0], NAME=[$1])
-        TableAccessRel(table=[[CATALOG, SALES, DEPT]])
+LogicalProject(DEPTNO=[$0], NAME=[$1])
+  Uncollect
+    Collect(field=[EXPR$0])
+      LogicalProject(DEPTNO=[$0], NAME=[$1])
+        LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
         </Resource>
         <Resource name="sql">
@@ -237,12 +237,12 @@ ProjectRel(DEPTNO=[$0], NAME=[$1])
     <TestCase name="testMultisetSubquery">
         <Resource name="plan">
             <![CDATA[
-ProjectRel(EXPR$0=[$1])
-  JoinRel(condition=[true], joinType=[inner])
-    ValuesRel(tuples=[[{ true }]])
-    CollectRel(field=[EXPR$0])
-      ProjectRel(DEPTNO=[$0])
-        TableAccessRel(table=[[CATALOG, SALES, DEPT]])
+LogicalProject(EXPR$0=[$1])
+  LogicalJoin(condition=[true], joinType=[inner])
+    LogicalValues(tuples=[[{ true }]])
+    Collect(field=[EXPR$0])
+      LogicalProject(DEPTNO=[$0])
+        LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
         </Resource>
         <Resource name="sql">
@@ -264,11 +264,11 @@ ProjectRel(EXPR$0=[$1])
     <TestCase name="testMultiset">
         <Resource name="plan">
             <![CDATA[
-ProjectRel(EXPR$0=['a'], EXPR$1=[$SLICE($2)])
-  JoinRel(condition=[true], joinType=[inner])
-    TableAccessRel(table=[[CATALOG, SALES, DEPT]])
-    CollectRel(field=[EXPR$0])
-      ValuesRel(tuples=[[{ 10 }]])
+LogicalProject(EXPR$0=['a'], EXPR$1=[$SLICE($2)])
+  LogicalJoin(condition=[true], joinType=[inner])
+    LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+    Collect(field=[EXPR$0])
+      LogicalValues(tuples=[[{ 10 }]])
 ]]>
         </Resource>
         <Resource name="sql">
@@ -289,15 +289,15 @@ ProjectRel(EXPR$0=[_ISO-8859-1'a'], EXPR$1=[$SLICE($2)])
     <TestCase name="testMultisetOfColumns">
         <Resource name="plan">
             <![CDATA[
-ProjectRel(EXPR$0=['abc'], EXPR$1=[$SLICE($9)])
-  CorrelatorRel(condition=[true], joinType=[inner], correlations=[[var0=offset7, var1=offset5]])
-    TableAccessRel(table=[[CATALOG, SALES, EMP]])
-    CollectRel(field=[EXPR$0])
-      UnionRel(all=[true])
-        ProjectRel(EXPR$0=[$cor0.DEPTNO])
-          OneRowRel
-        ProjectRel(EXPR$0=[$cor1.SAL])
-          OneRowRel
+LogicalProject(EXPR$0=['abc'], EXPR$1=[$SLICE($9)])
+  Correlator(condition=[true], joinType=[inner], correlations=[[var0=offset7, var1=offset5]])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    Collect(field=[EXPR$0])
+      LogicalUnion(all=[true])
+        LogicalProject(EXPR$0=[$cor0.DEPTNO])
+          LogicalOneRow
+        LogicalProject(EXPR$0=[$cor1.SAL])
+          LogicalOneRow
 ]]>
         </Resource>
         <Resource name="sql">
@@ -320,13 +320,13 @@ ProjectRel(EXPR$0=[_ISO-8859-1'abc'], EXPR$1=[$SLICE($8)])
     <TestCase name="testCorrelationJoin">
         <Resource name="plan">
             <![CDATA[
-ProjectRel(DEPTNO=[$0], NAME=[$1], EMPSET=[$2])
-  CorrelatorRel(condition=[true], joinType=[inner], correlations=[[var0=offset0]])
-    TableAccessRel(table=[[CATALOG, SALES, DEPT]])
-    CollectRel(field=[EXPR$0])
-      ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-        FilterRel(condition=[=($7, $cor0.DEPTNO)])
-          TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(DEPTNO=[$0], NAME=[$1], EMPSET=[$2])
+  Correlator(condition=[true], joinType=[inner], correlations=[[var0=offset0]])
+    LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+    Collect(field=[EXPR$0])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+        LogicalFilter(condition=[=($7, $cor0.DEPTNO)])
+          LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
         <Resource name="sql">
@@ -347,15 +347,15 @@ ProjectRel(DEPTNO=[$0], NAME=[$1], EMPSET=[$2])
     <TestCase name="testExists">
         <Resource name="plan">
             <![CDATA[
-ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-  FilterRel(condition=[IS NOT NULL($9)])
-    JoinRel(condition=[true], joinType=[left])
-      TableAccessRel(table=[[CATALOG, SALES, EMP]])
-      AggregateRel(group=[{}], agg#0=[MIN($0)])
-        ProjectRel($f0=[true])
-          ProjectRel(EXPR$0=[1])
-            FilterRel(condition=[=($0, 55)])
-              TableAccessRel(table=[[CATALOG, SALES, DEPT]])
+LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+  LogicalFilter(condition=[IS NOT NULL($9)])
+    LogicalJoin(condition=[true], joinType=[left])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+      LogicalAggregate(group=[{}], agg#0=[MIN($0)])
+        LogicalProject($f0=[true])
+          LogicalProject(EXPR$0=[1])
+            LogicalFilter(condition=[=($0, 55)])
+              LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
         </Resource>
         <Resource name="sql">
@@ -377,15 +377,15 @@ ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5],
     <TestCase name="testExistsCorrelated">
         <Resource name="plan">
             <![CDATA[
-ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-  FilterRel(condition=[IS NOT NULL($9)])
-    CorrelatorRel(condition=[true], joinType=[left], correlations=[[var0=offset7]])
-      TableAccessRel(table=[[CATALOG, SALES, EMP]])
-      AggregateRel(group=[{}], agg#0=[MIN($0)])
-        ProjectRel($f0=[true])
-          ProjectRel(EXPR$0=[1])
-            FilterRel(condition=[=($cor0.DEPTNO, $0)])
-              TableAccessRel(table=[[CATALOG, SALES, DEPT]])
+LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+  LogicalFilter(condition=[IS NOT NULL($9)])
+    Correlator(condition=[true], joinType=[left], correlations=[[var0=offset7]])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+      LogicalAggregate(group=[{}], agg#0=[MIN($0)])
+        LogicalProject($f0=[true])
+          LogicalProject(EXPR$0=[1])
+            LogicalFilter(condition=[=($cor0.DEPTNO, $0)])
+              LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
         </Resource>
         <Resource name="sql">
@@ -407,15 +407,15 @@ ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5],
     <TestCase name="testUnnestSelect">
         <Resource name="plan">
             <![CDATA[
-ProjectRel(EXPR$0=[$0])
-  UncollectRel
-    ProjectRel(EXPR$0=[$SLICE($2)])
-      CorrelatorRel(condition=[true], joinType=[inner], correlations=[[var0=offset0]])
-        TableAccessRel(table=[[CATALOG, SALES, DEPT]])
-        CollectRel(field=[EXPR$0])
-          UnionRel(all=[true])
-            ProjectRel(EXPR$0=[$cor0.DEPTNO])
-              OneRowRel
+LogicalProject(EXPR$0=[$0])
+  Uncollect
+    LogicalProject(EXPR$0=[$SLICE($2)])
+      Correlator(condition=[true], joinType=[inner], correlations=[[var0=offset0]])
+        LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+        Collect(field=[EXPR$0])
+          LogicalUnion(all=[true])
+            LogicalProject(EXPR$0=[$cor0.DEPTNO])
+              LogicalOneRow
 ]]>
         </Resource>
         <Resource name="sql">
@@ -439,12 +439,12 @@ ProjectRel(EXPR$0=[$0])
     <TestCase name="testLateral">
         <Resource name="plan">
             <![CDATA[
-ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$9], NAME=[$10])
-  CorrelatorRel(condition=[true], joinType=[inner], correlations=[[var0=offset7]])
-    TableAccessRel(table=[[CATALOG, SALES, EMP]])
-    ProjectRel(DEPTNO=[$0], NAME=[$1])
-      FilterRel(condition=[=($cor0.DEPTNO, $0)])
-        TableAccessRel(table=[[CATALOG, SALES, DEPT]])
+LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$9], NAME=[$10])
+  Correlator(condition=[true], joinType=[inner], correlations=[[var0=offset7]])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalProject(DEPTNO=[$0], NAME=[$1])
+      LogicalFilter(condition=[=($cor0.DEPTNO, $0)])
+        LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
         </Resource>
         <Resource name="sql">
@@ -464,11 +464,11 @@ ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5],
     <TestCase name="testElement">
         <Resource name="plan">
             <![CDATA[
-ProjectRel(EXPR$0=[ELEMENT($SLICE($9))])
-  JoinRel(condition=[true], joinType=[inner])
-    TableAccessRel(table=[[CATALOG, SALES, EMP]])
-    CollectRel(field=[EXPR$0])
-      ValuesRel(tuples=[[{ 5 }]])
+LogicalProject(EXPR$0=[ELEMENT($SLICE($9))])
+  LogicalJoin(condition=[true], joinType=[inner])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    Collect(field=[EXPR$0])
+      LogicalValues(tuples=[[{ 5 }]])
 ]]>
         </Resource>
         <Resource name="sql">
@@ -489,9 +489,9 @@ ProjectRel(EXPR$0=[ELEMENT($SLICE($8))])
     <TestCase name="testElementInValues">
         <Resource name="plan">
             <![CDATA[
-ProjectRel(EXPR$0=[ELEMENT($SLICE($0))])
-  CollectRel(field=[EXPR$0])
-    ValuesRel(tuples=[[{ 5 }]])
+LogicalProject(EXPR$0=[ELEMENT($SLICE($0))])
+  Collect(field=[EXPR$0])
+    LogicalValues(tuples=[[{ 5 }]])
 ]]>
         </Resource>
         <Resource name="sql">
@@ -511,11 +511,11 @@ ProjectRel(EXPR$0=[$0])
     <TestCase name="testUnionAll">
         <Resource name="plan">
             <![CDATA[
-UnionRel(all=[true])
-  ProjectRel(EMPNO=[$0])
-    TableAccessRel(table=[[CATALOG, SALES, EMP]])
-  ProjectRel(DEPTNO=[$0])
-    TableAccessRel(table=[[CATALOG, SALES, DEPT]])
+LogicalUnion(all=[true])
+  LogicalProject(EMPNO=[$0])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+  LogicalProject(DEPTNO=[$0])
+    LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
         </Resource>
         <Resource name="sql">
@@ -534,11 +534,11 @@ UnionRel(all=[true])
     <TestCase name="testUnion">
         <Resource name="plan">
             <![CDATA[
-UnionRel(all=[false])
-  ProjectRel(EMPNO=[$0])
-    TableAccessRel(table=[[CATALOG, SALES, EMP]])
-  ProjectRel(DEPTNO=[$0])
-    TableAccessRel(table=[[CATALOG, SALES, DEPT]])
+LogicalUnion(all=[false])
+  LogicalProject(EMPNO=[$0])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+  LogicalProject(DEPTNO=[$0])
+    LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
         </Resource>
         <Resource name="sql">
@@ -557,16 +557,16 @@ UnionRel(all=[false])
     <TestCase name="testUnionValues">
         <Resource name="plan">
             <![CDATA[
-UnionRel(all=[true])
-  UnionRel(all=[true])
-    ValuesRel(tuples=[[{ 10 }, { 20 }]])
-    ProjectRel(EXPR$0=[34])
-      TableAccessRel(table=[[CATALOG, SALES, EMP]])
-  UnionRel(all=[true])
-    ProjectRel(EXPR$0=[30])
-      OneRowRel
-    ProjectRel(EXPR$0=[+(45, 10)])
-      OneRowRel
+LogicalUnion(all=[true])
+  LogicalUnion(all=[true])
+    LogicalValues(tuples=[[{ 10 }, { 20 }]])
+    LogicalProject(EXPR$0=[34])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+  LogicalUnion(all=[true])
+    LogicalProject(EXPR$0=[30])
+      LogicalOneRow
+    LogicalProject(EXPR$0=[+(45, 10)])
+      LogicalOneRow
 ]]>
         </Resource>
         <Resource name="sql">
@@ -599,17 +599,17 @@ UnionRel(all=[true])
     <TestCase name="testUnionSubquery">
         <Resource name="plan">
             <![CDATA[
-ProjectRel(DEPTNO=[$7])
-  JoinRel(condition=[true], joinType=[inner])
-    TableAccessRel(table=[[CATALOG, SALES, EMP]])
-    UnionRel(all=[true])
-      UnionRel(all=[true])
-        ProjectRel(EMPNO=[$0])
-          TableAccessRel(table=[[CATALOG, SALES, EMP]])
-        ProjectRel(DEPTNO=[$0])
-          FilterRel(condition=[>($0, 20)])
-            TableAccessRel(table=[[CATALOG, SALES, DEPT]])
-      ValuesRel(tuples=[[{ 45 }, { 67 }]])
+LogicalProject(DEPTNO=[$7])
+  LogicalJoin(condition=[true], joinType=[inner])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalUnion(all=[true])
+      LogicalUnion(all=[true])
+        LogicalProject(EMPNO=[$0])
+          LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+        LogicalProject(DEPTNO=[$0])
+          LogicalFilter(condition=[>($0, 20)])
+            LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+      LogicalValues(tuples=[[{ 45 }, { 67 }]])
 ]]>
         </Resource>
         <Resource name="sql">
@@ -642,8 +642,8 @@ ProjectRel(DEPTNO=[$7])
     <TestCase name="testIsDistinctFrom">
         <Resource name="plan">
             <![CDATA[
-ProjectRel(EXPR$0=[CAST(CASE(IS NULL(1), IS NOT NULL(2), IS NULL(2), IS NOT NULL(1), <>(1, 2))):BOOLEAN NOT NULL])
-  ValuesRel(tuples=[[{ true }]])
+LogicalProject(EXPR$0=[CAST(CASE(IS NULL(1), IS NOT NULL(2), IS NULL(2), IS NOT NULL(1), <>(1, 2))):BOOLEAN NOT NULL])
+  LogicalValues(tuples=[[{ true }]])
 ]]>
         </Resource>
         <Resource name="sql">
@@ -661,8 +661,8 @@ ProjectRel(EXPR$0=[CASE(IS NULL(1), IS NOT NULL(2), IS NULL(2), IS NOT NULL(1),
     <TestCase name="testIsNotDistinctFrom">
         <Resource name="plan">
             <![CDATA[
-ProjectRel(EXPR$0=[CAST(CASE(IS NULL(1), IS NULL(2), IS NULL(2), IS NULL(1), =(1, 2))):BOOLEAN NOT NULL])
-  ValuesRel(tuples=[[{ true }]])
+LogicalProject(EXPR$0=[CAST(CASE(IS NULL(1), IS NULL(2), IS NULL(2), IS NULL(1), =(1, 2))):BOOLEAN NOT NULL])
+  LogicalValues(tuples=[[{ true }]])
 ]]>
         </Resource>
         <Resource name="sql">
@@ -680,8 +680,8 @@ ProjectRel(EXPR$0=[CASE(IS NULL(1), IS NULL(2), IS NULL(2), IS NULL(1), =(1, 2))
     <TestCase name="testNotLike">
         <Resource name="plan">
             <![CDATA[
-ProjectRel(EXPR$0=[NOT(LIKE('a', 'b', 'c'))])
-  OneRowRel
+LogicalProject(EXPR$0=[NOT(LIKE('a', 'b', 'c'))])
+  LogicalOneRow
 ]]>
         </Resource>
         <Resource name="sql">
@@ -698,9 +698,9 @@ ProjectRel(EXPR$0=[$0])
     <TestCase name="testOverMultiple">
         <Resource name="plan">
             <![CDATA[
-ProjectRel(EXPR$0=[CASE(>(COUNT($5) OVER (PARTITION BY $2 ORDER BY $4 ROWS BETWEEN 2 PRECEDING AND CURRENT ROW), 0), CAST($SUM0($5) OVER (PARTITION BY $2 ORDER BY $4 ROWS BETWEEN 2 PRECEDING AND CURRENT ROW)):INTEGER, null)], EXPR$1=[CASE(>(COUNT($7) OVER (PARTITION BY $2 ORDER BY $4 ROWS BETWEEN 2 PRECEDING AND CURRENT ROW), 0), CAST($SUM0($7) OVER (PARTITION BY $2 ORDER BY $4 ROWS BETWEEN 2 PRECEDING AND CURRENT ROW)):INTEGER, null)], EXPR$2=[CASE(>=(COUNT() OVER (PARTITION BY $2 ORDER BY $4 ROWS BETWEEN 3 PRECEDING AND CURRENT ROW), 2), CASE(>(COUNT($7) OVER (PARTITION BY $2 ORDER BY $4 ROWS BETWEEN 3 PRECEDING AND CURRENT ROW), 0), CAST($SUM0($7) OVER (PARTITION BY $2 ORDER BY $4 ROWS BETWEEN 3 PRECEDING AND CURRENT ROW)):INTEGER, null), null)])
-  FilterRel(condition=[>(-($7, $5), 999)])
-    TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(EXPR$0=[CASE(>(COUNT($5) OVER (PARTITION BY $2 ORDER BY $4 ROWS BETWEEN 2 PRECEDING AND CURRENT ROW), 0), CAST($SUM0($5) OVER (PARTITION BY $2 ORDER BY $4 ROWS BETWEEN 2 PRECEDING AND CURRENT ROW)):INTEGER, null)], EXPR$1=[CASE(>(COUNT($7) OVER (PARTITION BY $2 ORDER BY $4 ROWS BETWEEN 2 PRECEDING AND CURRENT ROW), 0), CAST($SUM0($7) OVER (PARTITION BY $2 ORDER BY $4 ROWS BETWEEN 2 PRECEDING AND CURRENT ROW)):INTEGER, null)], EXPR$2=[CASE(>=(COUNT() OVER (PARTITION BY $2 ORDER BY $4 ROWS BETWEEN 3 PRECEDING AND CURRENT ROW), 2), CASE(>(COUNT($7) OVER (PARTITION BY $2 ORDER BY $4 ROWS BETWEEN 3 PRECEDING AND CURRENT ROW), 0), CAST($SUM0($7) OVER (PARTITION BY $2 ORDER BY $4 ROWS BETWEEN 3 PRECEDING AND CURRENT ROW)):INTEGER, null), null)])
+  LogicalFilter(condition=[>(-($7, $5), 999)])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
         <Resource name="sql">
@@ -724,8 +724,8 @@ ProjectRel(EXPR$0=[SUM($5) OVER (PARTITION BY $2 ORDER BY $4 ROWS 2 PRECEDING)],
     <TestCase name="testCharLength">
         <Resource name="plan">
             <![CDATA[
-ProjectRel(EXPR$0=[CHAR_LENGTH('foo')])
-  OneRowRel
+LogicalProject(EXPR$0=[CHAR_LENGTH('foo')])
+  LogicalOneRow
 ]]>
         </Resource>
         <Resource name="sql">
@@ -742,8 +742,8 @@ ProjectRel(EXPR$0=[$0])
     <TestCase name="testOverAvg">
         <Resource name="plan">
             <![CDATA[
-ProjectRel(EXPR$0=[CASE(>(COUNT($5) OVER (PARTITION BY $2 ORDER BY $4 ROWS BETWEEN 2 PRECEDING AND CURRENT ROW), 0), CAST($SUM0($5) OVER (PARTITION BY $2 ORDER BY $4 ROWS BETWEEN 2 PRECEDING AND CURRENT ROW)):INTEGER, null)], EXPR$1=[CAST(/(CASE(>(COUNT($5) OVER (PARTITION BY $2 ORDER BY $4 ROWS BETWEEN 2 PRECEDING AND CURRENT ROW), 0), CAST($SUM0($5) OVER (PARTITION BY $2 ORDER BY $4 ROWS BETWEEN 2 PRECEDING AND CURRENT ROW)):INTEGER, null), COUNT($5) OVER (PARTITION BY $2 ORDER BY $4 ROWS BETWEEN 2 PRECEDING AND CURRENT ROW))):INTEGER])
-  TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(EXPR$0=[CASE(>(COUNT($5) OVER (PARTITION BY $2 ORDER BY $4 ROWS BETWEEN 2 PRECEDING AND CURRENT ROW), 0), CAST($SUM0($5) OVER (PARTITION BY $2 ORDER BY $4 ROWS BETWEEN 2 PRECEDING AND CURRENT ROW)):INTEGER, null)], EXPR$1=[CAST(/(CASE(>(COUNT($5) OVER (PARTITION BY $2 ORDER BY $4 ROWS BETWEEN 2 PRECEDING AND CURRENT ROW), 0), CAST($SUM0($5) OVER (PARTITION BY $2 ORDER BY $4 ROWS BETWEEN 2 PRECEDING AND CURRENT ROW)):INTEGER, null), COUNT($5) OVER (PARTITION BY $2 ORDER BY $4 ROWS BETWEEN 2 PRECEDING AND CURRENT ROW))):INTEGER])
+  LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
         <Resource name="sql">
@@ -762,8 +762,8 @@ ProjectRel(EXPR$0=[SUM($5) OVER (PARTITION BY $2 ORDER BY $4 ROWS 2 PRECEDING)],
     <TestCase name="testOverCountStar">
         <Resource name="plan">
             <![CDATA[
-ProjectRel(EXPR$0=[COUNT($5) OVER (PARTITION BY $2 ORDER BY $4 ROWS BETWEEN 2 PRECEDING AND CURRENT ROW)], EXPR$1=[COUNT() OVER (PARTITION BY $2 ORDER BY $4 ROWS BETWEEN 2 PRECEDING AND CURRENT ROW)])
-  TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(EXPR$0=[COUNT($5) OVER (PARTITION BY $2 ORDER BY $4 ROWS BETWEEN 2 PRECEDING AND CURRENT ROW)], EXPR$1=[COUNT() OVER (PARTITION BY $2 ORDER BY $4 ROWS BETWEEN 2 PRECEDING AND CURRENT ROW)])
+  LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
         <Resource name="sql">
@@ -785,8 +785,8 @@ ProjectRel(EXPR$0=[COUNT($5) OVER (PARTITION BY $2 ORDER BY $4 ROWS 2 PRECEDING)
         </Resource>
         <Resource name="plan">
             <![CDATA[
-ProjectRel(EXPR$0=[1])
-  TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(EXPR$0=[1])
+  LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -796,9 +796,9 @@ ProjectRel(EXPR$0=[1])
         </Resource>
         <Resource name="plan">
             <![CDATA[
-AggregateRel(group=[{0}])
-  ProjectRel(EXPR$0=[+($5, 5)])
-    TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalAggregate(group=[{0}])
+  LogicalProject(EXPR$0=[+($5, 5)])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -808,11 +808,11 @@ AggregateRel(group=[{0}])
         </Resource>
         <Resource name="plan">
             <![CDATA[
-AggregateRel(group=[{0}])
-  ProjectRel(EXPR$0=[$1])
-    AggregateRel(group=[{0}], EXPR$0=[SUM($1)])
-      ProjectRel(DEPTNO=[$7], SAL=[$5])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalAggregate(group=[{0}])
+  LogicalProject(EXPR$0=[$1])
+    LogicalAggregate(group=[{0}], EXPR$0=[SUM($1)])
+      LogicalProject(DEPTNO=[$7], SAL=[$5])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -822,8 +822,8 @@ AggregateRel(group=[{0}])
         </Resource>
         <Resource name="plan">
             <![CDATA[
-ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-  TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+  LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -833,8 +833,8 @@ ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5],
         </Resource>
         <Resource name="plan">
             <![CDATA[
-ProjectRel(I=[$0])
-  TableFunctionRel(invocation=[RAMP(3)], rowType=[RecordType(INTEGER I)])
+LogicalProject(I=[$0])
+  LogicalTableFunctionScan(invocation=[RAMP(3)], rowType=[RecordType(INTEGER I)])
 ]]>
         </Resource>
     </TestCase>
@@ -844,9 +844,9 @@ ProjectRel(I=[$0])
         </Resource>
         <Resource name="plan">
             <![CDATA[
-ProjectRel(EMPNO=[$0])
-  FilterRel(condition=[OR(=($7, 10), =($7, 20))])
-    TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(EMPNO=[$0])
+  LogicalFilter(condition=[OR(=($7, 10), =($7, 20))])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -856,12 +856,12 @@ ProjectRel(EMPNO=[$0])
         </Resource>
         <Resource name="plan">
             <![CDATA[
-ProjectRel(EMPNO=[$0])
-  JoinRel(condition=[=($9, $10)], joinType=[inner])
-    ProjectRel($f0=[$0], $f1=[$1], $f2=[$2], $f3=[$3], $f4=[$4], $f5=[$5], $f6=[$6], $f7=[$7], $f8=[$8], $f9=[$7])
-      TableAccessRel(table=[[CATALOG, SALES, EMP]])
-    AggregateRel(group=[{0}])
-      ValuesRel(tuples=[[{ 10 }, { 20 }, { 30 }, { 40 }, { 50 }, { 60 }, { 70 }, { 80 }, { 90 }, { 100 }, { 110 }, { 120 }, { 130 }, { 140 }, { 150 }, { 160 }, { 170 }, { 180 }, { 190 }, { 200 }, { 210 }, { 220 }, { 230 }]])
+LogicalProject(EMPNO=[$0])
+  LogicalJoin(condition=[=($9, $10)], joinType=[inner])
+    LogicalProject($f0=[$0], $f1=[$1], $f2=[$2], $f3=[$3], $f4=[$4], $f5=[$5], $f6=[$6], $f7=[$7], $f8=[$8], $f9=[$7])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalAggregate(group=[{0}])
+      LogicalValues(tuples=[[{ 10 }, { 20 }, { 30 }, { 40 }, { 50 }, { 60 }, { 70 }, { 80 }, { 90 }, { 100 }, { 110 }, { 120 }, { 130 }, { 140 }, { 150 }, { 160 }, { 170 }, { 180 }, { 190 }, { 200 }, { 210 }, { 220 }, { 230 }]])
 ]]>
         </Resource>
     </TestCase>
@@ -871,13 +871,13 @@ ProjectRel(EMPNO=[$0])
         </Resource>
         <Resource name="plan">
             <![CDATA[
-ProjectRel(EMPNO=[$0])
-  JoinRel(condition=[=($9, $10)], joinType=[inner])
-    ProjectRel($f0=[$0], $f1=[$1], $f2=[$2], $f3=[$3], $f4=[$4], $f5=[$5], $f6=[$6], $f7=[$7], $f8=[$8], $f9=[$7])
-      TableAccessRel(table=[[CATALOG, SALES, EMP]])
-    AggregateRel(group=[{0}])
-      ProjectRel(DEPTNO=[$0])
-        TableAccessRel(table=[[CATALOG, SALES, DEPT]])
+LogicalProject(EMPNO=[$0])
+  LogicalJoin(condition=[=($9, $10)], joinType=[inner])
+    LogicalProject($f0=[$0], $f1=[$1], $f2=[$2], $f3=[$3], $f4=[$4], $f5=[$5], $f6=[$6], $f7=[$7], $f8=[$8], $f9=[$7])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalAggregate(group=[{0}])
+      LogicalProject(DEPTNO=[$0])
+        LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
         </Resource>
     </TestCase>
@@ -887,12 +887,12 @@ ProjectRel(EMPNO=[$0])
         </Resource>
         <Resource name="plan">
             <![CDATA[
-ProjectRel(NAME=[$0])
-  TableFunctionRel(invocation=[DEDUP($0, $1, 'NAME')], rowType=[RecordType(VARCHAR(1024) NAME)])
-    ProjectRel(ENAME=[$1])
-      TableAccessRel(table=[[CATALOG, SALES, EMP]])
-    ProjectRel(NAME=[$1])
-      TableAccessRel(table=[[CATALOG, SALES, DEPT]])
+LogicalProject(NAME=[$0])
+  LogicalTableFunctionScan(invocation=[DEDUP($0, $1, 'NAME')], rowType=[RecordType(VARCHAR(1024) NAME)])
+    LogicalProject(ENAME=[$1])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalProject(NAME=[$1])
+      LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
         </Resource>
     </TestCase>
@@ -902,9 +902,9 @@ ProjectRel(NAME=[$0])
         </Resource>
         <Resource name="plan">
             <![CDATA[
-SortRel(sort0=[$0], dir0=[ASC])
-  ProjectRel(EMPNO=[$0])
-    TableAccessRel(table=[[CATALOG, SALES, EMP]])
+Sort(sort0=[$0], dir0=[ASC])
+  LogicalProject(EMPNO=[$0])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -914,9 +914,9 @@ SortRel(sort0=[$0], dir0=[ASC])
         </Resource>
         <Resource name="plan">
             <![CDATA[
-SortRel(sort0=[$0], dir0=[DESC-nulls-last])
-  ProjectRel(EMPNO=[$0])
-    TableAccessRel(table=[[CATALOG, SALES, EMP]])
+Sort(sort0=[$0], dir0=[DESC-nulls-last])
+  LogicalProject(EMPNO=[$0])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -926,21 +926,21 @@ SortRel(sort0=[$0], dir0=[DESC-nulls-last])
         </Resource>
         <Resource name="plan">
             <![CDATA[
-SortRel(sort0=[$1], dir0=[ASC])
-  ProjectRel(X=[+($0, 1)], Y=[-($0, 2)])
-    TableAccessRel(table=[[CATALOG, SALES, EMP]])
+Sort(sort0=[$1], dir0=[ASC])
+  LogicalProject(X=[+($0, 1)], Y=[-($0, 2)])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
     <TestCase name="testOrderByOrdinalDesc">
         <Resource name="sql">
-            <![CDATA[select empno + 1, deptno, empno from emp order by 2.5 desc]]>
+            <![CDATA[select empno + 1, deptno, empno from emp order by 2 desc]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
-SortRel(sort0=[$1], dir0=[DESC])
-  ProjectRel(EXPR$0=[+($0, 1)], DEPTNO=[$7], EMPNO=[$0])
-    TableAccessRel(table=[[CATALOG, SALES, EMP]])
+Sort(sort0=[$1], dir0=[DESC])
+  LogicalProject(EXPR$0=[+($0, 1)], DEPTNO=[$7], EMPNO=[$0])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -950,10 +950,10 @@ SortRel(sort0=[$1], dir0=[DESC])
         </Resource>
         <Resource name="plan">
             <![CDATA[
-ProjectRel(EXPR$0=[$0])
-  SortRel(sort0=[$1], sort1=[$0], dir0=[ASC], dir1=[DESC])
-    ProjectRel(EXPR$0=[+($0, 1)], DEPTNO=[$7])
-      TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(EXPR$0=[$0])
+  Sort(sort0=[$1], sort1=[$0], dir0=[ASC], dir1=[DESC])
+    LogicalProject(EXPR$0=[+($0, 1)], DEPTNO=[$7])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -963,10 +963,10 @@ ProjectRel(EXPR$0=[$0])
         </Resource>
         <Resource name="plan">
             <![CDATA[
-ProjectRel(EXPR$0=[$0], DEPTNO=[$1], EMPNO=[$2])
-  SortRel(sort0=[$3], dir0=[DESC])
-    ProjectRel(EXPR$0=[+($0, 1)], DEPTNO=[$7], EMPNO=[$0], EXPR$3=[-1])
-      TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(EXPR$0=[$0], DEPTNO=[$1], EMPNO=[$2])
+  Sort(sort0=[$3], dir0=[DESC])
+    LogicalProject(EXPR$0=[+($0, 1)], DEPTNO=[$7], EMPNO=[$0], EXPR$3=[-1])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -976,10 +976,10 @@ ProjectRel(EXPR$0=[$0], DEPTNO=[$1], EMPNO=[$2])
         </Resource>
         <Resource name="plan">
             <![CDATA[
-ProjectRel(EXPR$0=[$0], DEPTNO=[$1], EMPNO=[$2])
-  SortRel(sort0=[$3], dir0=[DESC])
-    ProjectRel(EXPR$0=[+($0, 1)], DEPTNO=[$7], EMPNO=[$0], EXPR$3=[+(1, 2)])
-      TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(EXPR$0=[$0], DEPTNO=[$1], EMPNO=[$2])
+  Sort(sort0=[$3], dir0=[DESC])
+    LogicalProject(EXPR$0=[+($0, 1)], DEPTNO=[$7], EMPNO=[$0], EXPR$3=[+(1, 2)])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -1002,10 +1002,10 @@ ProjectRel(EMPNO=[$0], Y=[$1])
         </Resource>
         <Resource name="plan">
             <![CDATA[
-ProjectRel(X=[$0], Y=[$1])
-  SortRel(sort0=[$2], dir0=[ASC])
-    ProjectRel(X=[+($0, 1)], Y=[-($0, 2)], EXPR$2=[+(-($0, 2), 3)])
-      TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(X=[$0], Y=[$1])
+  Sort(sort0=[$2], dir0=[ASC])
+    LogicalProject(X=[+($0, 1)], Y=[-($0, 2)], EXPR$2=[+(-($0, 2), 3)])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -1015,10 +1015,10 @@ ProjectRel(X=[$0], Y=[$1])
         </Resource>
         <Resource name="plan">
             <![CDATA[
-ProjectRel(EMPNO=[$0], Y=[$1])
-  SortRel(sort0=[$2], dir0=[ASC])
-    ProjectRel(EMPNO=[+($0, 1)], Y=[-($0, 2)], EXPR$2=[+(+($0, 1), 3)])
-      TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(EMPNO=[$0], Y=[$1])
+  Sort(sort0=[$2], dir0=[ASC])
+    LogicalProject(EMPNO=[+($0, 1)], Y=[-($0, 2)], EXPR$2=[+(+($0, 1), 3)])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -1028,13 +1028,13 @@ ProjectRel(EMPNO=[$0], Y=[$1])
         </Resource>
         <Resource name="plan">
             <![CDATA[
-SortRel(sort0=[$1], sort1=[$0], dir0=[DESC], dir1=[ASC])
-  ProjectRel(EMPNO=[$0], SAL=[$1])
-    UnionRel(all=[true])
-      ProjectRel(EMPNO=[$0], SAL=[$5])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
-      ProjectRel(DEPTNO=[$0], DEPTNO0=[$0])
-        TableAccessRel(table=[[CATALOG, SALES, DEPT]])
+Sort(sort0=[$1], sort1=[$0], dir0=[DESC], dir1=[ASC])
+  LogicalProject(EMPNO=[$0], SAL=[$1])
+    LogicalUnion(all=[true])
+      LogicalProject(EMPNO=[$0], SAL=[$5])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+      LogicalProject(DEPTNO=[$0], DEPTNO0=[$0])
+        LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
         </Resource>
     </TestCase>
@@ -1044,14 +1044,14 @@ SortRel(sort0=[$1], sort1=[$0], dir0=[DESC], dir1=[ASC])
         </Resource>
         <Resource name="plan">
             <![CDATA[
-ProjectRel(EMPNO=[$0], SAL=[$1])
-  SortRel(sort0=[$2], dir0=[ASC])
-    ProjectRel(EMPNO=[$0], SAL=[$1], EXPR$2=[+(*($0, $1), 2)])
-      UnionRel(all=[true])
-        ProjectRel(EMPNO=[$0], SAL=[$5])
-          TableAccessRel(table=[[CATALOG, SALES, EMP]])
-        ProjectRel(DEPTNO=[$0], DEPTNO0=[$0])
-          TableAccessRel(table=[[CATALOG, SALES, DEPT]])
+LogicalProject(EMPNO=[$0], SAL=[$1])
+  Sort(sort0=[$2], dir0=[ASC])
+    LogicalProject(EMPNO=[$0], SAL=[$1], EXPR$2=[+(*($0, $1), 2)])
+      LogicalUnion(all=[true])
+        LogicalProject(EMPNO=[$0], SAL=[$5])
+          LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+        LogicalProject(DEPTNO=[$0], DEPTNO0=[$0])
+          LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
         </Resource>
     </TestCase>
@@ -1061,12 +1061,12 @@ ProjectRel(EMPNO=[$0], SAL=[$1])
         </Resource>
         <Resource name="plan">
             <![CDATA[
-ProjectRel(DEPTNO=[$0], EXPR$1=[$1])
-  SortRel(sort0=[$2], sort1=[$3], dir0=[DESC], dir1=[ASC])
-    ProjectRel(DEPTNO=[$0], EXPR$1=[$1], EXPR$2=[*($0, $2)], EXPR$3=[$3])
-      AggregateRel(group=[{0}], EXPR$1=[COUNT()], agg#1=[SUM($1)], agg#2=[MIN($2)])
-        ProjectRel(DEPTNO=[$7], SAL=[$5], EMPNO=[$0])
-          TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(DEPTNO=[$0], EXPR$1=[$1])
+  Sort(sort0=[$2], sort1=[$3], dir0=[DESC], dir1=[ASC])
+    LogicalProject(DEPTNO=[$0], EXPR$1=[$1], EXPR$2=[*($0, $2)], EXPR$3=[$3])
+      LogicalAggregate(group=[{0}], EXPR$1=[COUNT()], agg#1=[SUM($1)], agg#2=[MIN($2)])
+        LogicalProject(DEPTNO=[$7], SAL=[$5], EMPNO=[$0])
+          LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -1076,11 +1076,11 @@ ProjectRel(DEPTNO=[$0], EXPR$1=[$1])
         </Resource>
         <Resource name="plan">
             <![CDATA[
-ProjectRel(EMPNO=[$0], EXPR$1=[$1])
-  SortRel(sort0=[$2], dir0=[ASC])
-    AggregateRel(group=[{0, 1, 2}])
-      ProjectRel(EMPNO=[$0], EXPR$1=[+($7, 1)], EXPR$2=[+(+($7, 1), $0)])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(EMPNO=[$0], EXPR$1=[$1])
+  Sort(sort0=[$2], dir0=[ASC])
+    LogicalAggregate(group=[{0, 1, 2}])
+      LogicalProject(EMPNO=[$0], EXPR$1=[+($7, 1)], EXPR$2=[+(+($7, 1), $0)])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -1090,12 +1090,12 @@ ProjectRel(EMPNO=[$0], EXPR$1=[$1])
         </Resource>
         <Resource name="plan">
             <![CDATA[
-ProjectRel(EMPNO=[$0])
-  SortRel(sort0=[$1], sort1=[$2], sort2=[$1], dir0=[DESC], dir1=[ASC], dir2=[ASC])
-    ProjectRel(EMPNO=[$0], EXPR$1=[+($5, $0)], EXPR$2=[*($5, $0)])
-      JoinRel(condition=[true], joinType=[inner])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
-        TableAccessRel(table=[[CATALOG, SALES, DEPT]])
+LogicalProject(EMPNO=[$0])
+  Sort(sort0=[$1], sort1=[$2], sort2=[$1], dir0=[DESC], dir1=[ASC], dir2=[ASC])
+    LogicalProject(EMPNO=[$0], EXPR$1=[+($5, $0)], EXPR$2=[*($5, $0)])
+      LogicalJoin(condition=[true], joinType=[inner])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+        LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
         </Resource>
     </TestCase>
@@ -1105,13 +1105,13 @@ ProjectRel(EMPNO=[$0])
         </Resource>
         <Resource name="plan">
             <![CDATA[
-SortRel(sort0=[$1], dir0=[ASC])
-  ProjectRel(EMPNO=[$0], SAL=[$1])
-    UnionRel(all=[true])
-      ProjectRel(EMPNO=[$0], SAL=[$5])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
-      ProjectRel(DEPTNO=[$0], DEPTNO0=[$0])
-        TableAccessRel(table=[[CATALOG, SALES, DEPT]])
+Sort(sort0=[$1], dir0=[ASC])
+  LogicalProject(EMPNO=[$0], SAL=[$1])
+    LogicalUnion(all=[true])
+      LogicalProject(EMPNO=[$0], SAL=[$5])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+      LogicalProject(DEPTNO=[$0], DEPTNO0=[$0])
+        LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
         </Resource>
     </TestCase>
@@ -1121,9 +1121,9 @@ SortRel(sort0=[$1], dir0=[ASC])
         </Resource>
         <Resource name="plan">
             <![CDATA[
-ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-  FilterRel(condition=[>($0, 5)])
-    TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+  LogicalFilter(condition=[>($0, 5)])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -1137,38 +1137,38 @@ where empno > 5]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
-ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$9], NAME=[$10])
-  FilterRel(condition=[>($0, 5)])
-    ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$9], NAME=[$10])
-      JoinRel(condition=[=($7, $9)], joinType=[inner])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
-        TableAccessRel(table=[[CATALOG, SALES, DEPT]])
+LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$9], NAME=[$10])
+  LogicalFilter(condition=[>($0, 5)])
+    LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$9], NAME=[$10])
+      LogicalJoin(condition=[=($7, $9)], joinType=[inner])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+        LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
         </Resource>
     </TestCase>
     <TestCase name="testOverOrderWindow">
         <Resource name="sql">
-            <![CDATA[select last_value(deptno) over (order by empno)
+            <![CDATA[select last_value(deptno) over w
 from emp
-]]>
+window w as (order by empno)]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
-ProjectRel(EXPR$0=[LAST_VALUE($7) OVER (ORDER BY $0 RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)])
-  TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(EXPR$0=[LAST_VALUE($7) OVER (ORDER BY $0 RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)])
+  LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
     <TestCase name="testOverOrderFollowingWindow">
         <Resource name="sql">
-            <![CDATA[select last_value(deptno) over (order by empno rows 2 following)
+            <![CDATA[select last_value(deptno) over w
 from emp
-]]>
+window w as (order by empno rows 2 following)]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
-ProjectRel(EXPR$0=[LAST_VALUE($7) OVER (ORDER BY $0 ROWS BETWEEN CURRENT ROW AND 2 FOLLOWING)])
-  TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(EXPR$0=[LAST_VALUE($7) OVER (ORDER BY $0 ROWS BETWEEN CURRENT ROW AND 2 FOLLOWING)])
+  LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -1193,8 +1193,8 @@ window w1 as (partition by job order by hiredate rows 2 preceding)]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
-ProjectRel(EXPR$0=[CASE(>(COUNT($5) OVER (PARTITION BY $2 ORDER BY $4 ROWS BETWEEN 2 PRECEDING AND CURRENT ROW), 0), CAST($SUM0($5) OVER (PARTITION BY $2 ORDER BY $4 ROWS BETWEEN 2 PRECEDING AND CURRENT ROW)):INTEGER, null)], EXPR$1=[/(CASE(>(COUNT(CAST($5):REAL NOT NULL) OVER (PARTITION BY $2 ORDER BY $4 ROWS BETWEEN 2 PRECEDING AND CURRENT ROW), 0), CAST($SUM0(CAST($5):REAL NOT NULL) OVER (PARTITION BY $2 ORDER BY $4 ROWS BETWEEN 2 PRECEDING AND CURRENT ROW)):REAL, null), COUNT(CAST($5):REAL NOT NULL) OVER (PARTITION BY $2 ORDER BY $4 ROWS BETWEEN 2 PRECEDING AND CURRENT ROW))])
-  TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(EXPR$0=[CASE(>(COUNT($5) OVER (PARTITION BY $2 ORDER BY $4 ROWS BETWEEN 2 PRECEDING AND CURRENT ROW), 0), CAST($SUM0($5) OVER (PARTITION BY $2 ORDER BY $4 ROWS BETWEEN 2 PRECEDING AND CURRENT ROW)):INTEGER, null)], EXPR$1=[/(CASE(>(COUNT(CAST($5):REAL NOT NULL) OVER (PARTITION BY $2 ORDER BY $4 ROWS BETWEEN 2 PRECEDING AND CURRENT ROW), 0), CAST($SUM0(CAST($5):REAL NOT NULL) OVER (PARTITION BY $2 ORDER BY $4 ROWS BETWEEN 2 PRECEDING AND CURRENT ROW)):REAL, null), COUNT(CAST($5):REAL NOT NULL) OVER (PARTITION BY $2 ORDER BY $4 ROWS BETWEEN 2 PRECEDING AND CURRENT ROW))])
+  LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -1204,10 +1204,10 @@ ProjectRel(EXPR$0=[CASE(>(COUNT($5) OVER (PARTITION BY $2 ORDER BY $4 ROWS BETWE
         </Resource>
         <Resource name="plan">
             <![CDATA[
-ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$9], NAME=[$10])
-  JoinRel(condition=[=($7, $9)], joinType=[inner])
-    TableAccessRel(table=[[CATALOG, SALES, EMP]])
-    TableAccessRel(table=[[CATALOG, SALES, DEPT]])
+LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$9], NAME=[$10])
+  LogicalJoin(condition=[=($7, $9)], joinType=[inner])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
         </Resource>
     </TestCase>
@@ -1217,11 +1217,11 @@ ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5],
         </Resource>
         <Resource name="plan">
             <![CDATA[
-ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$9], NAME=[$10], EMPNO0=[$11])
-  JoinRel(condition=[AND(=($7, $9), =($0, $11))], joinType=[left])
-    TableAccessRel(table=[[CATALOG, SALES, EMP]])
-    ProjectRel(DEPTNO=[$0], NAME=[$1], EMPNO=[*($0, 5)])
-      TableAccessRel(table=[[CATALOG, SALES, DEPT]])
+LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$9], NAME=[$10], EMPNO0=[$11])
+  LogicalJoin(condition=[AND(=($7, $9), =($0, $11))], joinType=[left])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalProject(DEPTNO=[$0], NAME=[$1], EMPNO=[*($0, 5)])
+      LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
         </Resource>
     </TestCase>
@@ -1231,10 +1231,10 @@ ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5],
         </Resource>
         <Resource name="plan">
             <![CDATA[
-ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$9], NAME=[$10])
-  JoinRel(condition=[=($7, $9)], joinType=[inner])
-    TableAccessRel(table=[[CATALOG, SALES, EMP]])
-    TableAccessRel(table=[[CATALOG, SALES, DEPT]])
+LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$9], NAME=[$10])
+  LogicalJoin(condition=[=($7, $9)], joinType=[inner])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
         </Resource>
     </TestCase>
@@ -1245,10 +1245,10 @@ ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5],
         </Resource>
         <Resource name="plan">
             <![CDATA[
-ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$9], NAME=[$10])
-  JoinRel(condition=[AND(=($7, $9), OR(=($0, 1), =($0, 3)))], joinType=[inner])
-    TableAccessRel(table=[[CATALOG, SALES, EMP]])
-    TableAccessRel(table=[[CATALOG, SALES, DEPT]])
+LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$9], NAME=[$10])
+  LogicalJoin(condition=[AND(=($7, $9), OR(=($0, 1), =($0, 3)))], joinType=[inner])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
         </Resource>
     </TestCase>
@@ -1258,10 +1258,10 @@ ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5],
         </Resource>
         <Resource name="plan">
             <![CDATA[
-ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-  FilterRel(condition=[>($0, 5)])
-    SamplingRel(mode=[bernoulli], rate=[0.5], repeatableSeed=[-])
-      TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+  LogicalFilter(condition=[>($0, 5)])
+    Sample(mode=[bernoulli], rate=[0.5], repeatableSeed=[-])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -1275,14 +1275,14 @@ where empno > 5]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
-ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$9], NAME=[$10])
-  FilterRel(condition=[>($0, 5)])
-    SamplingRel(mode=[bernoulli], rate=[0.5], repeatableSeed=[99])
-      ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$9], NAME=[$10])
-        JoinRel(condition=[=($7, $9)], joinType=[inner])
-          SamplingRel(mode=[bernoulli], rate=[0.1], repeatableSeed=[1])
-            TableAccessRel(table=[[CATALOG, SALES, EMP]])
-          TableAccessRel(table=[[CATALOG, SALES, DEPT]])
+LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$9], NAME=[$10])
+  LogicalFilter(condition=[>($0, 5)])
+    Sample(mode=[bernoulli], rate=[0.5], repeatableSeed=[99])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$9], NAME=[$10])
+        LogicalJoin(condition=[=($7, $9)], joinType=[inner])
+          Sample(mode=[bernoulli], rate=[0.1], repeatableSeed=[1])
+            LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+          LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
         </Resource>
     </TestCase>
@@ -1292,10 +1292,10 @@ ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5],
         </Resource>
         <Resource name="plan">
             <![CDATA[
-ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-  FilterRel(condition=[>($0, 5)])
-    SamplingRel(mode=[system], rate=[0.5], repeatableSeed=[-])
-      TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+  LogicalFilter(condition=[>($0, 5)])
+    Sample(mode=[system], rate=[0.5], repeatableSeed=[-])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -1309,14 +1309,14 @@ where empno > 5]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
-ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$9], NAME=[$10])
-  FilterRel(condition=[>($0, 5)])
-    SamplingRel(mode=[system], rate=[0.5], repeatableSeed=[99])
-      ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$9], NAME=[$10])
-        JoinRel(condition=[=($7, $9)], joinType=[inner])
-          SamplingRel(mode=[system], rate=[0.1], repeatableSeed=[1])
-            TableAccessRel(table=[[CATALOG, SALES, EMP]])
-          TableAccessRel(table=[[CATALOG, SALES, DEPT]])
+LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$9], NAME=[$10])
+  LogicalFilter(condition=[>($0, 5)])
+    Sample(mode=[system], rate=[0.5], repeatableSeed=[99])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$9], NAME=[$10])
+        LogicalJoin(condition=[=($7, $9)], joinType=[inner])
+          Sample(mode=[system], rate=[0.1], repeatableSeed=[1])
+            LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+          LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
         </Resource>
     </TestCase>
@@ -1326,12 +1326,12 @@ ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5],
         </Resource>
         <Resource name="plan">
             <![CDATA[
-ProjectRel(EXPR$0=[$0], DEPTNO=[$1], EXPR$2=[$0])
-  AggregateRel(group=[{0, 1}])
-    ProjectRel(EXPR$0=[$0], DEPTNO=[$1])
-      ProjectRel(EXPR$0=[+($5, 5)], DEPTNO=[$7], EXPR$2=[+($5, 5)])
-        FilterRel(condition=[<($7, 10)])
-          TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(EXPR$0=[$0], DEPTNO=[$1], EXPR$2=[$0])
+  LogicalAggregate(group=[{0, 1}])
+    LogicalProject(EXPR$0=[$0], DEPTNO=[$1])
+      LogicalProject(EXPR$0=[+($5, 5)], DEPTNO=[$7], EXPR$2=[+($5, 5)])
+        LogicalFilter(condition=[<($7, 10)])
+          LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -1343,10 +1343,10 @@ where empno > 10]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
-AggregateRel(group=[{}], EXPR$0=[COUNT()], EXPR$1=[SUM($0)])
-  ProjectRel(SAL=[$5])
-    FilterRel(condition=[>($0, 10)])
-      TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalAggregate(group=[{}], EXPR$0=[COUNT()], EXPR$1=[SUM($0)])
+  LogicalProject(SAL=[$5])
+    LogicalFilter(condition=[>($0, 10)])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -1359,10 +1359,10 @@ where c like 'X%']]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
-ProjectRel(EXPR$0=[+($0, $1)])
-  FilterRel(condition=[LIKE($2, 'X%')])
-    ProjectRel(DEPTNO=[$0], ONE=[1], NAME=[$1])
-      TableAccessRel(table=[[CATALOG, SALES, DEPT]])
+LogicalProject(EXPR$0=[+($0, $1)])
+  LogicalFilter(condition=[LIKE($2, 'X%')])
+    LogicalProject(DEPTNO=[$0], ONE=[1], NAME=[$1])
+      LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
         </Resource>
     </TestCase>
@@ -1375,13 +1375,13 @@ order by c + a]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
-ProjectRel(A=[$0], B=[$1], C=[$2], DEPTNO=[$3], NAME=[$4])
-  SortRel(sort0=[$5], dir0=[ASC])
-    ProjectRel(A=[$0], B=[$1], C=[$2], DEPTNO=[$3], NAME=[$4], EXPR$5=[+($2, $0)])
-      JoinRel(condition=[=($3, $2)], joinType=[inner])
-        ProjectRel(A=[$2], B=[$1], C=[$0])
-          ValuesRel(tuples=[[{ 1, 2, 3 }]])
-        TableAccessRel(table=[[CATALOG, SALES, DEPT]])
+LogicalProject(A=[$0], B=[$1], C=[$2], DEPTNO=[$3], NAME=[$4])
+  Sort(sort0=[$5], dir0=[ASC])
+    LogicalProject(A=[$0], B=[$1], C=[$2], DEPTNO=[$3], NAME=[$4], EXPR$5=[+($2, $0)])
+      LogicalJoin(condition=[=($3, $2)], joinType=[inner])
+        LogicalProject(A=[$2], B=[$1], C=[$0])
+          LogicalValues(tuples=[[{ 1, 2, 3 }]])
+        LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
         </Resource>
     </TestCase>
@@ -1391,14 +1391,14 @@ ProjectRel(A=[$0], B=[$1], C=[$2], DEPTNO=[$3], NAME=[$4])
         </Resource>
         <Resource name="plan">
             <![CDATA[
-ProjectRel(GRADE=[$1])
-  JoinRel(condition=[true], joinType=[inner])
-    UnionRel(all=[false])
-      ProjectRel(EMPNO=[$0])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
-      ProjectRel(DEPTNO=[$0])
-        TableAccessRel(table=[[CATALOG, SALES, DEPT]])
-    TableAccessRel(table=[[CATALOG, SALES, SALGRADE]])
+LogicalProject(GRADE=[$1])
+  LogicalJoin(condition=[true], joinType=[inner])
+    LogicalUnion(all=[false])
+      LogicalProject(EMPNO=[$0])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+      LogicalProject(DEPTNO=[$0])
+        LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+    LogicalTableScan(table=[[CATALOG, SALES, SALGRADE]])
 ]]>
         </Resource>
     </TestCase>
@@ -1408,10 +1408,10 @@ ProjectRel(GRADE=[$1])
         </Resource>
         <Resource name="plan">
             <![CDATA[
-ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$9], NAME=[$10])
-  JoinRel(condition=[=($7, $9)], joinType=[inner])
-    TableAccessRel(table=[[CATALOG, SALES, EMP]])
-    TableAccessRel(table=[[CATALOG, SALES, DEPT]])
+LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$9], NAME=[$10])
+  LogicalJoin(condition=[=($7, $9)], joinType=[inner])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
         </Resource>
     </TestCase>
@@ -1421,11 +1421,11 @@ ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5],
         </Resource>
         <Resource name="plan">
             <![CDATA[
-ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], FOO=[$9], NAME=[$10])
-  JoinRel(condition=[true], joinType=[inner])
-    TableAccessRel(table=[[CATALOG, SALES, EMP]])
-    ProjectRel(FOO=[$0], NAME=[$1])
-      TableAccessRel(table=[[CATALOG, SALES, DEPT]])
+LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], FOO=[$9], NAME=[$10])
+  LogicalJoin(condition=[true], joinType=[inner])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalProject(FOO=[$0], NAME=[$1])
+      LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
         </Resource>
     </TestCase>
@@ -1435,11 +1435,11 @@ ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5],
         </Resource>
         <Resource name="plan">
             <![CDATA[
-ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$9], ENAME0=[$10])
-  JoinRel(condition=[AND(=($1, $10), =($7, $9))], joinType=[inner])
-    TableAccessRel(table=[[CATALOG, SALES, EMP]])
-    ProjectRel(DEPTNO=[$0], ENAME=[$1])
-      TableAccessRel(table=[[CATALOG, SALES, DEPT]])
+LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$9], ENAME0=[$10])
+  LogicalJoin(condition=[AND(=($1, $10), =($7, $9))], joinType=[inner])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalProject(DEPTNO=[$0], ENAME=[$1])
+      LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
         </Resource>
     </TestCase>
@@ -1449,9 +1449,9 @@ ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5],
         </Resource>
         <Resource name="plan">
             <![CDATA[
-SortRel(sort0=[$0], dir0=[ASC], offset=[10], fetch=[5])
-  ProjectRel(EMPNO=[$0])
-    TableAccessRel(table=[[CATALOG, SALES, EMP]])
+Sort(sort0=[$0], dir0=[ASC], offset=[10], fetch=[5])
+  LogicalProject(EMPNO=[$0])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -1461,9 +1461,9 @@ SortRel(sort0=[$0], dir0=[ASC], offset=[10], fetch=[5])
         </Resource>
         <Resource name="plan">
             <![CDATA[
-SortRel(fetch=[5])
-  ProjectRel(EMPNO=[$0])
-    TableAccessRel(table=[[CATALOG, SALES, EMP]])
+Sort(fetch=[5])
+  LogicalProject(EMPNO=[$0])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -1473,9 +1473,9 @@ SortRel(fetch=[5])
         </Resource>
         <Resource name="plan">
             <![CDATA[
-SortRel(offset=[10], fetch=[5])
-  ProjectRel(EMPNO=[$0])
-    TableAccessRel(table=[[CATALOG, SALES, EMP]])
+Sort(offset=[10], fetch=[5])
+  LogicalProject(EMPNO=[$0])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -1485,9 +1485,9 @@ SortRel(offset=[10], fetch=[5])
         </Resource>
         <Resource name="plan">
             <![CDATA[
-SortRel(offset=[10])
-  ProjectRel(EMPNO=[$0])
-    TableAccessRel(table=[[CATALOG, SALES, EMP]])
+Sort(offset=[10])
+  LogicalProject(EMPNO=[$0])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -1500,9 +1500,9 @@ and (deptno = 8 or empno < 100)]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
-ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-  FilterRel(condition=[AND(<($7, 10), >($7, 5), OR(=($7, 8), <($0, 100)))])
-    TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+  LogicalFilter(condition=[AND(<($7, 10), >($7, 5), OR(=($7, 8), <($0, 100)))])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -1515,12 +1515,12 @@ join emp as e2 using (empno)]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
-ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$9], NAME=[$10], EMPNO0=[$11], ENAME0=[$12], JOB0=[$13], MGR0=[$14], HIREDATE0=[$15], SAL0=[$16], COMM0=[$17], DEPTNO1=[$18], SLACKER0=[$19])
-  JoinRel(condition=[=($0, $11)], joinType=[inner])
-    JoinRel(condition=[=($7, $9)], joinType=[inner])
-      TableAccessRel(table=[[CATALOG, SALES, EMP]])
-      TableAccessRel(table=[[CATALOG, SALES, DEPT]])
-    TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$9], NAME=[$10], EMPNO0=[$11], ENAME0=[$12], JOB0=[$13], MGR0=[$14], HIREDATE0=[$15], SAL0=[$16], COMM0=[$17], DEPTNO1=[$18], SLACKER0=[$19])
+  LogicalJoin(condition=[=($0, $11)], joinType=[inner])
+    LogicalJoin(condition=[=($7, $9)], joinType=[inner])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+      LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -1530,13 +1530,13 @@ ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5],
         </Resource>
         <Resource name="plan">
             <![CDATA[
-ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$9], NAME=[$10])
-  ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$10], NAME=[$11])
-    JoinRel(condition=[=($9, $12)], joinType=[inner])
-      ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], $f9=[+($7, 1)])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
-      ProjectRel(DEPTNO=[$0], NAME=[$1], $f2=[-($0, 2)])
-        TableAccessRel(table=[[CATALOG, SALES, DEPT]])
+LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$9], NAME=[$10])
+  LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$10], NAME=[$11])
+    LogicalJoin(condition=[=($9, $12)], joinType=[inner])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], $f9=[+($7, 1)])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+      LogicalProject(DEPTNO=[$0], NAME=[$1], $f2=[-($0, 2)])
+        LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
         </Resource>
     </TestCase>
@@ -1547,9 +1547,9 @@ select empno as "e", deptno as d, 1 as "e" from EMP)]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
-ProjectRel(e=[$0])
-  ProjectRel(e=[$0], D=[$7], e0=[1])
-    TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(e=[$0])
+  LogicalProject(e=[$0], D=[$7], e0=[1])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -1560,9 +1560,9 @@ select * from emp2]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
-ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-  ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-    TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+  LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -1573,14 +1573,14 @@ select empno from emp2 where deptno < 30 union all select deptno from emp]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
-UnionRel(all=[true])
-  ProjectRel(EMPNO=[$0])
-    FilterRel(condition=[<($7, 30)])
-      ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-        FilterRel(condition=[>($7, 10)])
-          TableAccessRel(table=[[CATALOG, SALES, EMP]])
-  ProjectRel(DEPTNO=[$7])
-    TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalUnion(all=[true])
+  LogicalProject(EMPNO=[$0])
+    LogicalFilter(condition=[<($7, 30)])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+        LogicalFilter(condition=[>($7, 10)])
+          LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+  LogicalProject(DEPTNO=[$7])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -1593,17 +1593,17 @@ where exists (
         </Resource>
         <Resource name="plan">
             <![CDATA[
-ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-  FilterRel(condition=[IS NOT NULL($9)])
-    CorrelatorRel(condition=[true], joinType=[left], correlations=[[var1=offset7, var0=offset7]])
-      TableAccessRel(table=[[CATALOG, SALES, EMP]])
-      AggregateRel(group=[{}], agg#0=[MIN($0)])
-        ProjectRel($f0=[true])
-          ProjectRel(EXPR$0=[1])
-            FilterRel(condition=[<=($0, $cor1.DEPTNO)])
-              ProjectRel(DEPTNO=[$0], NAME=[$1])
-                FilterRel(condition=[>=($0, $cor0.DEPTNO)])
-                  TableAccessRel(table=[[CATALOG, SALES, DEPT]])
+LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+  LogicalFilter(condition=[IS NOT NULL($9)])
+    Correlator(condition=[true], joinType=[left], correlations=[[var1=offset7, var0=offset7]])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+      LogicalAggregate(group=[{}], agg#0=[MIN($0)])
+        LogicalProject($f0=[true])
+          LogicalProject(EXPR$0=[1])
+            LogicalFilter(condition=[<=($0, $cor1.DEPTNO)])
+              LogicalProject(DEPTNO=[$0], NAME=[$1])
+                LogicalFilter(condition=[>=($0, $cor0.DEPTNO)])
+                  LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
         </Resource>
     </TestCase>
@@ -1615,15 +1615,15 @@ from emp]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
-ProjectRel(C=[$9])
-  JoinRel(condition=[true], joinType=[left])
-    TableAccessRel(table=[[CATALOG, SALES, EMP]])
-    AggregateRel(group=[{}], agg#0=[SINGLE_VALUE($0)])
-      AggregateRel(group=[{}], EXPR$0=[COUNT()])
-        ProjectRel($f0=[0])
-          ProjectRel(DEPTNO=[$0], NAME=[$1])
-            FilterRel(condition=[>($0, 10)])
-              TableAccessRel(table=[[CATALOG, SALES, DEPT]])
+LogicalProject(C=[$9])
+  LogicalJoin(condition=[true], joinType=[left])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalAggregate(group=[{}], agg#0=[SINGLE_VALUE($0)])
+      LogicalAggregate(group=[{}], EXPR$0=[COUNT()])
+        LogicalProject($f0=[0])
+          LogicalProject(DEPTNO=[$0], NAME=[$1])
+            LogicalFilter(condition=[>($0, 10)])
+              LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
         </Resource>
     </TestCase>
@@ -1633,9 +1633,9 @@ ProjectRel(C=[$9])
         </Resource>
         <Resource name="plan">
             <![CDATA[
-AggregateRel(group=[{}], EXPR$0=[SUM($0)])
-  ProjectRel(DEPTNO=[$7])
-    TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalAggregate(group=[{}], EXPR$0=[SUM($0)])
+  LogicalProject(DEPTNO=[$7])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -1645,12 +1645,12 @@ AggregateRel(group=[{}], EXPR$0=[SUM($0)])
         </Resource>
         <Resource name="plan">
             <![CDATA[
-ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$9], NAME=[$10])
-  ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$10], NAME=[$11])
-    JoinRel(condition=[=($10, $9)], joinType=[inner])
-      ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], $f9=[+($7, 0)])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
-      TableAccessRel(table=[[CATALOG, SALES, DEPT]])
+LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$9], NAME=[$10])
+  LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$10], NAME=[$11])
+    LogicalJoin(condition=[=($10, $9)], joinType=[inner])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], $f9=[+($7, 0)])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+      LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
         </Resource>
     </TestCase>
@@ -1660,12 +1660,12 @@ ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5],
         </Resource>
         <Resource name="plan">
             <![CDATA[
-ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$9], NAME=[$10])
-  ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$10], NAME=[$11])
-    JoinRel(condition=[=($9, $10)], joinType=[inner])
-      ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], $f9=[+($7, 0)])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
-      TableAccessRel(table=[[CATALOG, SALES, DEPT]])
+LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$9], NAME=[$10])
+  LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$10], NAME=[$11])
+    LogicalJoin(condition=[=($9, $10)], joinType=[inner])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], $f9=[+($7, 0)])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+      LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
         </Resource>
     </TestCase>
@@ -1675,16 +1675,16 @@ ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5],
         </Resource>
         <Resource name="plan">
             <![CDATA[
-ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$9], NAME=[$10])
-  JoinRel(condition=[=($7, $11)], joinType=[inner])
-    TableAccessRel(table=[[CATALOG, SALES, EMP]])
-    ProjectRel(DEPTNO=[$0], NAME=[$1], $f0=[$2])
-      FilterRel(condition=[=($2, $0)])
-        JoinRel(condition=[true], joinType=[inner])
-          TableAccessRel(table=[[CATALOG, SALES, DEPT]])
-          AggregateRel(group=[{0}])
-            ProjectRel($f0=[$7])
-              TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$9], NAME=[$10])
+  LogicalJoin(condition=[=($7, $11)], joinType=[inner])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalProject(DEPTNO=[$0], NAME=[$1], $f0=[$2])
+      LogicalFilter(condition=[=($2, $0)])
+        LogicalJoin(condition=[true], joinType=[inner])
+          LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+          LogicalAggregate(group=[{0}])
+            LogicalProject($f0=[$7])
+              LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -1697,27 +1697,27 @@ where exists (
         </Resource>
         <Resource name="plan">
             <![CDATA[
-ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-  FilterRel(condition=[IS NOT NULL($9)])
-    ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], $f0=[$11])
-      JoinRel(condition=[AND(=($7, $10), =($7, $9))], joinType=[left])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
-        AggregateRel(group=[{0, 1}], agg#0=[MIN($2)])
-          ProjectRel($f01=[$1], $f00=[$2], $f0=[$0])
-            ProjectRel($f0=[true], $f01=[$1], $f00=[$2])
-              ProjectRel(EXPR$0=[1], $f0=[$2], $f00=[$3])
-                FilterRel(condition=[<=($0, $3)])
-                  JoinRel(condition=[true], joinType=[inner])
-                    ProjectRel(DEPTNO=[$0], NAME=[$1], $f0=[$2])
-                      FilterRel(condition=[>=($0, $2)])
-                        JoinRel(condition=[true], joinType=[inner])
-                          TableAccessRel(table=[[CATALOG, SALES, DEPT]])
-                          AggregateRel(group=[{0}])
-                            ProjectRel($f0=[$7])
-                              TableAccessRel(table=[[CATALOG, SALES, EMP]])
-                    AggregateRel(group=[{0}])
-                      ProjectRel($f0=[$7])
-                        TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+  LogicalFilter(condition=[IS NOT NULL($9)])
+    LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], $f0=[$11])
+      LogicalJoin(condition=[AND(=($7, $10), =($7, $9))], joinType=[left])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+        LogicalAggregate(group=[{0, 1}], agg#0=[MIN($2)])
+          LogicalProject($f01=[$1], $f00=[$2], $f0=[$0])
+            LogicalProject($f0=[true], $f01=[$1], $f00=[$2])
+              LogicalProject(EXPR$0=[1], $f0=[$2], $f00=[$3])
+                LogicalFilter(condition=[<=($0, $3)])
+                  LogicalJoin(condition=[true], joinType=[inner])
+                    LogicalProject(DEPTNO=[$0], NAME=[$1], $f0=[$2])
+                      LogicalFilter(condition=[>=($0, $2)])
+                        LogicalJoin(condition=[true], joinType=[inner])
+                          LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+                          LogicalAggregate(group=[{0}])
+                            LogicalProject($f0=[$7])
+                              LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+                    LogicalAggregate(group=[{0}])
+                      LogicalProject($f0=[$7])
+                        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -1727,21 +1727,21 @@ ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5],
         </Resource>
         <Resource name="plan">
             <![CDATA[
-ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-  FilterRel(condition=[IS NOT NULL($9)])
-    ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], $f0=[$10])
-      JoinRel(condition=[=($7, $9)], joinType=[left])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
-        AggregateRel(group=[{0}], agg#0=[MIN($1)])
-          ProjectRel($f01=[$1], $f0=[$0])
-            ProjectRel($f0=[true], $f01=[$1])
-              ProjectRel(EXPR$0=[1], $f0=[$2])
-                FilterRel(condition=[=($2, $0)])
-                  JoinRel(condition=[true], joinType=[inner])
-                    TableAccessRel(table=[[CATALOG, SALES, DEPT]])
-                    AggregateRel(group=[{0}])
-                      ProjectRel($f0=[$7])
-                        TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+  LogicalFilter(condition=[IS NOT NULL($9)])
+    LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], $f0=[$10])
+      LogicalJoin(condition=[=($7, $9)], joinType=[left])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+        LogicalAggregate(group=[{0}], agg#0=[MIN($1)])
+          LogicalProject($f01=[$1], $f0=[$0])
+            LogicalProject($f0=[true], $f01=[$1])
+              LogicalProject(EXPR$0=[1], $f0=[$2])
+                LogicalFilter(condition=[=($2, $0)])
+                  LogicalJoin(condition=[true], joinType=[inner])
+                    LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+                    LogicalAggregate(group=[{0}])
+                      LogicalProject($f0=[$7])
+                        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -1752,16 +1752,16 @@ ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5],
         </Resource>
         <Resource name="plan">
             <![CDATA[
-ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-  FilterRel(condition=[IS NOT NULL($9)])
-    CorrelatorRel(condition=[true], joinType=[left], correlations=[[var0=offset7]])
-      TableAccessRel(table=[[CATALOG, SALES, EMP]])
-      AggregateRel(group=[{}], agg#0=[MIN($0)])
-        ProjectRel($f0=[true])
-          SortRel(fetch=[1])
-            ProjectRel(EXPR$0=[1])
-              FilterRel(condition=[=($cor0.DEPTNO, $0)])
-                TableAccessRel(table=[[CATALOG, SALES, DEPT]])
+LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+  LogicalFilter(condition=[IS NOT NULL($9)])
+    Correlator(condition=[true], joinType=[left], correlations=[[var0=offset7]])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+      LogicalAggregate(group=[{}], agg#0=[MIN($0)])
+        LogicalProject($f0=[true])
+          Sort(fetch=[1])
+            LogicalProject(EXPR$0=[1])
+              LogicalFilter(condition=[=($cor0.DEPTNO, $0)])
+                LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
         </Resource>
     </TestCase>
@@ -1772,17 +1772,17 @@ ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5],
         </Resource>
         <Resource name="plan">
             <![CDATA[
-ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-  FilterRel(condition=[IS NOT NULL($9)])
-    ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], $f0=[$9])
-      CorrelatorRel(condition=[true], joinType=[left], correlations=[[var0=offset7]])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
-        AggregateRel(group=[{}], agg#0=[MIN($0)])
-          ProjectRel($f0=[true])
-            SortRel(fetch=[1])
-              ProjectRel(EXPR$0=[1])
-                FilterRel(condition=[=($cor0.DEPTNO, $0)])
-                  TableAccessRel(table=[[CATALOG, SALES, DEPT]])
+LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+  LogicalFilter(condition=[IS NOT NULL($9)])
+    LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], $f0=[$9])
+      Correlator(condition=[true], joinType=[left], correlations=[[var0=offset7]])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+        LogicalAggregate(group=[{}], agg#0=[MIN($0)])
+          LogicalProject($f0=[true])
+            Sort(fetch=[1])
+              LogicalProject(EXPR$0=[1])
+                LogicalFilter(condition=[=($cor0.DEPTNO, $0)])
+                  LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
         </Resource>
     </TestCase>
@@ -1792,15 +1792,15 @@ ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5],
         </Resource>
         <Resource name="plan">
             <![CDATA[
-ProjectRel(DEPTNO=[$0], NAME=[$1], EXPR$0=[$2])
-  CorrelatorRel(condition=[true], joinType=[inner], correlations=[[var0=offset0]])
-    TableAccessRel(table=[[CATALOG, SALES, DEPT]])
-    UncollectRel
-      ProjectRel(EXPR$0=[$SLICE($0)])
-        CollectRel(field=[EXPR$0])
-          UnionRel(all=[true])
-            ProjectRel(EXPR$0=[*($cor0.DEPTNO, 2)])
-              OneRowRel
+LogicalProject(DEPTNO=[$0], NAME=[$1], EXPR$0=[$2])
+  Correlator(condition=[true], joinType=[inner], correlations=[[var0=offset0]])
+    LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+    Uncollect
+      LogicalProject(EXPR$0=[$SLICE($0)])
+        Collect(field=[EXPR$0])
+          LogicalUnion(all=[true])
+            LogicalProject(EXPR$0=[*($cor0.DEPTNO, 2)])
+              LogicalOneRow
 ]]>
         </Resource>
     </TestCase>
@@ -1811,14 +1811,14 @@ select x from w where x < 30 union all select deptno from dept]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
-UnionRel(all=[true])
-  ProjectRel(X=[$0])
-    FilterRel(condition=[<($0, 30)])
-      ProjectRel(DEPTNO=[$0], NAME=[$1])
-        FilterRel(condition=[>($0, 10)])
-          TableAccessRel(table=[[CATALOG, SALES, DEPT]])
-  ProjectRel(DEPTNO=[$0])
-    TableAccessRel(table=[[CATALOG, SALES, DEPT]])
+LogicalUnion(all=[true])
+  LogicalProject(X=[$0])
+    LogicalFilter(condition=[<($0, 30)])
+      LogicalProject(DEPTNO=[$0], NAME=[$1])
+        LogicalFilter(condition=[>($0, 10)])
+          LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+  LogicalProject(DEPTNO=[$0])
+    LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
         </Resource>
     </TestCase>
@@ -1829,11 +1829,11 @@ select * from emp2 order by deptno]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
-SortRel(sort0=[$7], dir0=[ASC])
-  ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-    ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-      ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
+Sort(sort0=[$7], dir0=[ASC])
+  LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+    LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -1847,16 +1847,16 @@ order by empno + x]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
-ProjectRel(EMPNO=[$0], X=[$1])
-  SortRel(sort0=[$2], dir0=[ASC])
-    ProjectRel(EMPNO=[$0], X=[$1], EXPR$2=[+($0, $1)])
-      UnionRel(all=[true])
-        ProjectRel(EMPNO=[$0], X=[$1])
-          ProjectRel(EMPNO=[$0], X=[$7])
-            TableAccessRel(table=[[CATALOG, SALES, EMP]])
-        ProjectRel(EMPNO=[$0], X=[$1])
-          ProjectRel(EMPNO=[$0], X=[$7])
-            TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(EMPNO=[$0], X=[$1])
+  Sort(sort0=[$2], dir0=[ASC])
+    LogicalProject(EMPNO=[$0], X=[$1], EXPR$2=[+($0, $1)])
+      LogicalUnion(all=[true])
+        LogicalProject(EMPNO=[$0], X=[$1])
+          LogicalProject(EMPNO=[$0], X=[$7])
+            LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+        LogicalProject(EMPNO=[$0], X=[$1])
+          LogicalProject(EMPNO=[$0], X=[$7])
+            LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -1868,19 +1868,19 @@ from dept]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
-ProjectRel(NAME=[$1], EXPR$1=[CASE(=($2, 0), false, IS NOT NULL($6), true, IS NULL($4), null, <($3, $2), null, false)])
-  JoinRel(condition=[=($4, $5)], joinType=[left])
-    ProjectRel($f0=[$0], $f1=[$1], $f2=[$2], $f3=[$3], $f4=[$0])
-      JoinRel(condition=[true], joinType=[inner])
-        TableAccessRel(table=[[CATALOG, SALES, DEPT]])
-        AggregateRel(group=[{}], agg#0=[COUNT()], agg#1=[COUNT($0)])
-          ProjectRel($f0=[$0], $f1=[true])
-            ProjectRel(EXPR$0=[CASE(true, CAST($7):INTEGER, null)])
-              TableAccessRel(table=[[CATALOG, SALES, EMP]])
-    AggregateRel(group=[{0}], agg#0=[MIN($1)])
-      ProjectRel($f0=[$0], $f1=[true])
-        ProjectRel(EXPR$0=[CASE(true, CAST($7):INTEGER, null)])
-          TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(NAME=[$1], EXPR$1=[CASE(=($2, 0), false, IS NOT NULL($6), true, IS NULL($4), null, <($3, $2), null, false)])
+  LogicalJoin(condition=[=($4, $5)], joinType=[left])
+    LogicalProject($f0=[$0], $f1=[$1], $f2=[$2], $f3=[$3], $f4=[$0])
+      LogicalJoin(condition=[true], joinType=[inner])
+        LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+        LogicalAggregate(group=[{}], agg#0=[COUNT()], agg#1=[COUNT($0)])
+          LogicalProject($f0=[$0], $f1=[true])
+            LogicalProject(EXPR$0=[CASE(true, CAST($7):INTEGER, null)])
+              LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalAggregate(group=[{0}], agg#0=[MIN($1)])
+      LogicalProject($f0=[$0], $f1=[true])
+        LogicalProject(EXPR$0=[CASE(true, CAST($7):INTEGER, null)])
+          LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -1892,19 +1892,19 @@ from emp]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
-ProjectRel(EMPNO=[$0], EXPR$1=[NOT(CASE(=($9, 0), false, IS NOT NULL($13), true, IS NULL($11), null, <($10, $9), null, false))])
-  JoinRel(condition=[=($11, $12)], joinType=[left])
-    ProjectRel($f0=[$0], $f1=[$1], $f2=[$2], $f3=[$3], $f4=[$4], $f5=[$5], $f6=[$6], $f7=[$7], $f8=[$8], $f9=[$9], $f10=[$10], $f11=[$7])
-      JoinRel(condition=[true], joinType=[inner])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
-        AggregateRel(group=[{}], agg#0=[COUNT()], agg#1=[COUNT($0)])
-          ProjectRel($f0=[$0], $f1=[true])
-            ProjectRel(EXPR$0=[CASE(true, CAST($0):INTEGER, null)])
-              TableAccessRel(table=[[CATALOG, SALES, DEPT]])
-    AggregateRel(group=[{0}], agg#0=[MIN($1)])
-      ProjectRel($f0=[$0], $f1=[true])
-        ProjectRel(EXPR$0=[CASE(true, CAST($0):INTEGER, null)])
-          TableAccessRel(table=[[CATALOG, SALES, DEPT]])
+LogicalProject(EMPNO=[$0], EXPR$1=[NOT(CASE(=($9, 0), false, IS NOT NULL($13), true, IS NULL($11), null, <($10, $9), null, false))])
+  LogicalJoin(condition=[=($11, $12)], joinType=[left])
+    LogicalProject($f0=[$0], $f1=[$1], $f2=[$2], $f3=[$3], $f4=[$4], $f5=[$5], $f6=[$6], $f7=[$7], $f8=[$8], $f9=[$9], $f10=[$10], $f11=[$7])
+      LogicalJoin(condition=[true], joinType=[inner])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+        LogicalAggregate(group=[{}], agg#0=[COUNT()], agg#1=[COUNT($0)])
+          LogicalProject($f0=[$0], $f1=[true])
+            LogicalProject(EXPR$0=[CASE(true, CAST($0):INTEGER, null)])
+              LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+    LogicalAggregate(group=[{0}], agg#0=[MIN($1)])
+      LogicalProject($f0=[$0], $f1=[true])
+        LogicalProject(EXPR$0=[CASE(true, CAST($0):INTEGER, null)])
+          LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
         </Resource>
     </TestCase>
@@ -1914,20 +1914,20 @@ ProjectRel(EMPNO=[$0], EXPR$1=[NOT(CASE(=($9, 0), false, IS NOT NULL($13), true,
         </Resource>
         <Resource name="plan">
             <![CDATA[
-ProjectRel(EMPNO=[$0])
-  FilterRel(condition=[NOT(CASE(=($9, 0), false, IS NOT NULL($13), true, IS NULL($11), null, <($10, $9), null, false))])
-

<TRUNCATED>

[13/58] [abbrv] [partial] incubator-calcite git commit: [CALCITE-306] Standardize code style for "import package.*; "

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java b/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
index 7c7472d..63038fe 100644
--- a/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
@@ -14,63 +14,60 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.test;
-
-import java.util.List;
-
-import org.eigenbase.rel.RelNode;
-import org.eigenbase.rel.TableModificationRel;
-import org.eigenbase.rel.metadata.CachingRelMetadataProvider;
-import org.eigenbase.rel.metadata.ChainedRelMetadataProvider;
-import org.eigenbase.rel.metadata.DefaultRelMetadataProvider;
-import org.eigenbase.rel.metadata.RelMetadataProvider;
-import org.eigenbase.rel.rules.AddRedundantSemiJoinRule;
-import org.eigenbase.rel.rules.AggregateProjectMergeRule;
-import org.eigenbase.rel.rules.CoerceInputsRule;
-import org.eigenbase.rel.rules.ConvertMultiJoinRule;
-import org.eigenbase.rel.rules.ExtractJoinFilterRule;
-import org.eigenbase.rel.rules.FilterAggregateTransposeRule;
-import org.eigenbase.rel.rules.FilterToCalcRule;
-import org.eigenbase.rel.rules.MergeCalcRule;
-import org.eigenbase.rel.rules.MergeProjectRule;
-import org.eigenbase.rel.rules.ProjectToCalcRule;
-import org.eigenbase.rel.rules.PullConstantsThroughAggregatesRule;
-import org.eigenbase.rel.rules.PushAggregateThroughUnionRule;
-import org.eigenbase.rel.rules.PushFilterPastJoinRule;
-import org.eigenbase.rel.rules.PushFilterPastProjectRule;
-import org.eigenbase.rel.rules.PushFilterPastSetOpRule;
-import org.eigenbase.rel.rules.PushJoinThroughUnionRule;
-import org.eigenbase.rel.rules.PushProjectPastFilterRule;
-import org.eigenbase.rel.rules.PushProjectPastJoinRule;
-import org.eigenbase.rel.rules.PushProjectPastSetOpRule;
-import org.eigenbase.rel.rules.PushSemiJoinPastFilterRule;
-import org.eigenbase.rel.rules.PushSemiJoinPastJoinRule;
-import org.eigenbase.rel.rules.PushSemiJoinPastProjectRule;
-import org.eigenbase.rel.rules.ReduceAggregatesRule;
-import org.eigenbase.rel.rules.ReduceExpressionsRule;
-import org.eigenbase.rel.rules.ReduceValuesRule;
-import org.eigenbase.rel.rules.RemoveDistinctAggregateRule;
-import org.eigenbase.rel.rules.RemoveEmptyRules;
-import org.eigenbase.rel.rules.RemoveSemiJoinRule;
-import org.eigenbase.rel.rules.RemoveTrivialProjectRule;
-import org.eigenbase.rel.rules.SemiJoinRule;
-import org.eigenbase.rel.rules.TableAccessRule;
-import org.eigenbase.rel.rules.TransitivePredicatesOnJoinRule;
-import org.eigenbase.rel.rules.UnionToDistinctRule;
-import org.eigenbase.relopt.RelOptUtil;
-import org.eigenbase.relopt.hep.HepMatchOrder;
-import org.eigenbase.relopt.hep.HepPlanner;
-import org.eigenbase.relopt.hep.HepProgram;
-import org.eigenbase.relopt.hep.HepProgramBuilder;
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.reltype.RelDataTypeFactory;
-import org.eigenbase.sql.SqlNode;
-import org.eigenbase.sql.type.SqlTypeName;
-import org.eigenbase.sql.validate.SqlValidator;
-import org.eigenbase.sql2rel.SqlToRelConverter;
-import org.eigenbase.util.Util;
-
-import net.hydromatic.optiq.prepare.Prepare;
+package org.apache.calcite.test;
+
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.hep.HepMatchOrder;
+import org.apache.calcite.plan.hep.HepPlanner;
+import org.apache.calcite.plan.hep.HepProgram;
+import org.apache.calcite.plan.hep.HepProgramBuilder;
+import org.apache.calcite.prepare.Prepare;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.logical.LogicalTableModify;
+import org.apache.calcite.rel.metadata.CachingRelMetadataProvider;
+import org.apache.calcite.rel.metadata.ChainedRelMetadataProvider;
+import org.apache.calcite.rel.metadata.DefaultRelMetadataProvider;
+import org.apache.calcite.rel.metadata.RelMetadataProvider;
+import org.apache.calcite.rel.rules.AggregateExpandDistinctAggregatesRule;
+import org.apache.calcite.rel.rules.AggregateProjectMergeRule;
+import org.apache.calcite.rel.rules.AggregateProjectPullUpConstantsRule;
+import org.apache.calcite.rel.rules.AggregateReduceFunctionsRule;
+import org.apache.calcite.rel.rules.AggregateUnionTransposeRule;
+import org.apache.calcite.rel.rules.CalcMergeRule;
+import org.apache.calcite.rel.rules.CoerceInputsRule;
+import org.apache.calcite.rel.rules.EmptyPruneRules;
+import org.apache.calcite.rel.rules.FilterAggregateTransposeRule;
+import org.apache.calcite.rel.rules.FilterJoinRule;
+import org.apache.calcite.rel.rules.FilterProjectTransposeRule;
+import org.apache.calcite.rel.rules.FilterSetOpTransposeRule;
+import org.apache.calcite.rel.rules.FilterToCalcRule;
+import org.apache.calcite.rel.rules.JoinAddRedundantSemiJoinRule;
+import org.apache.calcite.rel.rules.JoinExtractFilterRule;
+import org.apache.calcite.rel.rules.JoinPushTransitivePredicatesRule;
+import org.apache.calcite.rel.rules.JoinToMultiJoinRule;
+import org.apache.calcite.rel.rules.JoinUnionTransposeRule;
+import org.apache.calcite.rel.rules.ProjectFilterTransposeRule;
+import org.apache.calcite.rel.rules.ProjectJoinTransposeRule;
+import org.apache.calcite.rel.rules.ProjectMergeRule;
+import org.apache.calcite.rel.rules.ProjectRemoveRule;
+import org.apache.calcite.rel.rules.ProjectSetOpTransposeRule;
+import org.apache.calcite.rel.rules.ProjectToCalcRule;
+import org.apache.calcite.rel.rules.ReduceExpressionsRule;
+import org.apache.calcite.rel.rules.SemiJoinFilterTransposeRule;
+import org.apache.calcite.rel.rules.SemiJoinJoinTransposeRule;
+import org.apache.calcite.rel.rules.SemiJoinProjectTransposeRule;
+import org.apache.calcite.rel.rules.SemiJoinRemoveRule;
+import org.apache.calcite.rel.rules.SemiJoinRule;
+import org.apache.calcite.rel.rules.TableScanRule;
+import org.apache.calcite.rel.rules.UnionToDistinctRule;
+import org.apache.calcite.rel.rules.ValuesReduceRule;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql2rel.SqlToRelConverter;
+import org.apache.calcite.util.Util;
 
 import com.google.common.base.Function;
 import com.google.common.collect.Lists;
@@ -78,14 +75,16 @@ import com.google.common.collect.Lists;
 import org.junit.Ignore;
 import org.junit.Test;
 
+import java.util.List;
+
 import static org.junit.Assert.assertTrue;
 
 /**
- * Unit test for rules in {@code org.eigenbase.rel} and subpackages.
+ * Unit test for rules in {@code org.apache.calcite.rel} and subpackages.
  *
  * <p>As input, the test supplies a SQL statement and a single rule; the SQL is
  * translated into relational algebra and then fed into a
- * {@link org.eigenbase.relopt.hep.HepPlanner}. The planner fires the rule on
+ * {@link org.apache.calcite.plan.hep.HepPlanner}. The planner fires the rule on
  * every
  * pattern match in a depth-first left-to-right preorder traversal of the tree
  * for as long as the rule continues to succeed in applying its transform. (For
@@ -128,20 +127,17 @@ public class RelOptRulesTest extends RelOptTestBase {
   }
 
   @Test public void testUnionToDistinctRule() {
-    checkPlanning(
-        UnionToDistinctRule.INSTANCE,
+    checkPlanning(UnionToDistinctRule.INSTANCE,
         "select * from dept union select * from dept");
   }
 
   @Test public void testExtractJoinFilterRule() {
-    checkPlanning(
-        ExtractJoinFilterRule.INSTANCE,
+    checkPlanning(JoinExtractFilterRule.INSTANCE,
         "select 1 from emp inner join dept on emp.deptno=dept.deptno");
   }
 
   @Test public void testAddRedundantSemiJoinRule() {
-    checkPlanning(
-        AddRedundantSemiJoinRule.INSTANCE,
+    checkPlanning(JoinAddRedundantSemiJoinRule.INSTANCE,
         "select 1 from emp inner join dept on emp.deptno = dept.deptno");
   }
 
@@ -151,68 +147,62 @@ public class RelOptRulesTest extends RelOptTestBase {
     // NULL.
     final HepProgram preProgram =
         HepProgram.builder()
-            .addRuleInstance(MergeProjectRule.INSTANCE)
-            .addRuleInstance(PushFilterPastProjectRule.INSTANCE)
+            .addRuleInstance(ProjectMergeRule.INSTANCE)
+            .addRuleInstance(FilterProjectTransposeRule.INSTANCE)
             .build();
     final HepProgram program =
         HepProgram.builder()
-            .addRuleInstance(PushFilterPastJoinRule.FILTER_ON_JOIN)
+            .addRuleInstance(FilterJoinRule.FILTER_ON_JOIN)
             .build();
     checkPlanning(tester.withDecorrelation(true).withTrim(true), preProgram,
         new HepPlanner(program),
         "select * from dept where exists (\n"
-        + "  select * from emp\n"
-        + "  where emp.deptno = dept.deptno\n"
-        + "  and emp.sal > 100)");
+            + "  select * from emp\n"
+            + "  where emp.deptno = dept.deptno\n"
+            + "  and emp.sal > 100)");
   }
 
   @Test public void testFullOuterJoinSimplificationToLeftOuter() {
-    checkPlanning(
-        PushFilterPastJoinRule.FILTER_ON_JOIN,
+    checkPlanning(FilterJoinRule.FILTER_ON_JOIN,
         "select 1 from sales.dept d full outer join sales.emp e"
-        + " on d.deptno = e.deptno"
-        + " where d.name = 'Charlie'");
+            + " on d.deptno = e.deptno"
+            + " where d.name = 'Charlie'");
   }
 
   @Test public void testFullOuterJoinSimplificationToRightOuter() {
-    checkPlanning(
-        PushFilterPastJoinRule.FILTER_ON_JOIN,
+    checkPlanning(FilterJoinRule.FILTER_ON_JOIN,
         "select 1 from sales.dept d full outer join sales.emp e"
-        + " on d.deptno = e.deptno"
-        + " where e.sal > 100");
+            + " on d.deptno = e.deptno"
+            + " where e.sal > 100");
   }
 
   @Test public void testFullOuterJoinSimplificationToInner() {
-    checkPlanning(
-        PushFilterPastJoinRule.FILTER_ON_JOIN,
+    checkPlanning(FilterJoinRule.FILTER_ON_JOIN,
         "select 1 from sales.dept d full outer join sales.emp e"
-        + " on d.deptno = e.deptno"
-        + " where d.name = 'Charlie' and e.sal > 100");
+            + " on d.deptno = e.deptno"
+            + " where d.name = 'Charlie' and e.sal > 100");
   }
 
   @Test public void testLeftOuterJoinSimplificationToInner() {
-    checkPlanning(
-        PushFilterPastJoinRule.FILTER_ON_JOIN,
+    checkPlanning(FilterJoinRule.FILTER_ON_JOIN,
         "select 1 from sales.dept d left outer join sales.emp e"
-        + " on d.deptno = e.deptno"
-        + " where e.sal > 100");
+            + " on d.deptno = e.deptno"
+            + " where e.sal > 100");
   }
 
 
   @Test public void testRightOuterJoinSimplificationToInner() {
-    checkPlanning(
-        PushFilterPastJoinRule.FILTER_ON_JOIN,
+    checkPlanning(FilterJoinRule.FILTER_ON_JOIN,
         "select 1 from sales.dept d right outer join sales.emp e"
-        + " on d.deptno = e.deptno"
-        + " where d.name = 'Charlie'");
+            + " on d.deptno = e.deptno"
+            + " where d.name = 'Charlie'");
   }
 
   @Test public void testPushFilterPastAgg() {
-    checkPlanning(
-        FilterAggregateTransposeRule.INSTANCE,
+    checkPlanning(FilterAggregateTransposeRule.INSTANCE,
         "select dname, c from"
-        + " (select name dname, count(*) as c from dept group by name) t"
-        + " where dname = 'Charlie'");
+            + " (select name dname, count(*) as c from dept group by name) t"
+            + " where dname = 'Charlie'");
   }
 
   /** Test case for
@@ -221,17 +211,17 @@ public class RelOptRulesTest extends RelOptTestBase {
   @Test public void testPushFilterPastAggTwo() {
     checkPlanning(FilterAggregateTransposeRule.INSTANCE,
         "select dept1.c1 from (\n"
-        + "  select dept.name as c1, count(*) as c2\n"
-        + "  from dept where dept.name > 'b' group by dept.name) dept1\n"
-        + "where dept1.c1 > 'c' and (dept1.c2 > 30 or dept1.c1 < 'z')");
+            + "  select dept.name as c1, count(*) as c2\n"
+            + "  from dept where dept.name > 'b' group by dept.name) dept1\n"
+            + "where dept1.c1 > 'c' and (dept1.c2 > 30 or dept1.c1 < 'z')");
   }
 
   @Test public void testSemiJoinRule() {
     final HepProgram preProgram =
         HepProgram.builder()
-            .addRuleInstance(PushFilterPastProjectRule.INSTANCE)
-            .addRuleInstance(PushFilterPastJoinRule.FILTER_ON_JOIN)
-            .addRuleInstance(MergeProjectRule.INSTANCE)
+            .addRuleInstance(FilterProjectTransposeRule.INSTANCE)
+            .addRuleInstance(FilterJoinRule.FILTER_ON_JOIN)
+            .addRuleInstance(ProjectMergeRule.INSTANCE)
             .build();
     final HepProgram program =
         HepProgram.builder()
@@ -247,7 +237,7 @@ public class RelOptRulesTest extends RelOptTestBase {
 
   /** Test case for
    * <a href="https://issues.apache.org/jira/browse/CALCITE-438">[CALCITE-438],
-   * Push predicates through SemiJoinRel</a>. */
+   * Push predicates through SemiJoin</a>. */
   @Test public void testPushFilterThroughSemiJoin() {
     final HepProgram preProgram =
         HepProgram.builder()
@@ -256,9 +246,9 @@ public class RelOptRulesTest extends RelOptTestBase {
 
     final HepProgram program =
         HepProgram.builder()
-            .addRuleInstance(PushFilterPastProjectRule.INSTANCE)
-            .addRuleInstance(PushFilterPastJoinRule.FILTER_ON_JOIN)
-            .addRuleInstance(PushFilterPastJoinRule.JOIN)
+            .addRuleInstance(FilterProjectTransposeRule.INSTANCE)
+            .addRuleInstance(FilterJoinRule.FILTER_ON_JOIN)
+            .addRuleInstance(FilterJoinRule.JOIN)
             .build();
     checkPlanning(tester.withDecorrelation(true).withTrim(false), preProgram,
         new HepPlanner(program),
@@ -298,9 +288,9 @@ public class RelOptRulesTest extends RelOptTestBase {
 
     final HepProgram program =
         HepProgram.builder()
-            .addRuleInstance(PushFilterPastProjectRule.INSTANCE)
-            .addRuleInstance(PushFilterPastJoinRule.FILTER_ON_JOIN)
-            .addRuleInstance(MergeProjectRule.INSTANCE)
+            .addRuleInstance(FilterProjectTransposeRule.INSTANCE)
+            .addRuleInstance(FilterJoinRule.FILTER_ON_JOIN)
+            .addRuleInstance(ProjectMergeRule.INSTANCE)
             .addRuleInstance(SemiJoinRule.INSTANCE)
             .build();
 
@@ -321,66 +311,60 @@ public class RelOptRulesTest extends RelOptTestBase {
   }
 
   @Test public void testReduceAverage() {
-    checkPlanning(
-        ReduceAggregatesRule.INSTANCE,
+    checkPlanning(AggregateReduceFunctionsRule.INSTANCE,
         "select name, max(name), avg(deptno), min(name)"
-        + " from sales.dept group by name");
+            + " from sales.dept group by name");
   }
 
   @Test public void testDistinctCount() {
     final HepProgram program = HepProgram.builder()
-        .addRuleInstance(RemoveDistinctAggregateRule.INSTANCE)
+        .addRuleInstance(AggregateExpandDistinctAggregatesRule.INSTANCE)
         .addRuleInstance(AggregateProjectMergeRule.INSTANCE)
         .build();
     checkPlanning(program,
         "select deptno, count(distinct ename)"
-        + " from sales.emp group by deptno");
+            + " from sales.emp group by deptno");
   }
 
   @Test public void testPushProjectPastFilter() {
-    checkPlanning(
-        PushProjectPastFilterRule.INSTANCE,
+    checkPlanning(ProjectFilterTransposeRule.INSTANCE,
         "select empno + deptno from emp where sal = 10 * comm "
-        + "and upper(ename) = 'FOO'");
+            + "and upper(ename) = 'FOO'");
   }
 
   @Test public void testPushProjectPastJoin() {
-    checkPlanning(
-        PushProjectPastJoinRule.INSTANCE,
+    checkPlanning(ProjectJoinTransposeRule.INSTANCE,
         "select e.sal + b.comm from emp e inner join bonus b "
-        + "on e.ename = b.ename and e.deptno = 10");
+            + "on e.ename = b.ename and e.deptno = 10");
   }
 
   @Test public void testPushProjectPastSetOp() {
-    checkPlanning(
-        PushProjectPastSetOpRule.INSTANCE,
+    checkPlanning(ProjectSetOpTransposeRule.INSTANCE,
         "select sal from "
-        + "(select * from emp e1 union all select * from emp e2)");
+            + "(select * from emp e1 union all select * from emp e2)");
   }
 
   @Test public void testPushJoinThroughUnionOnLeft() {
-    checkPlanning(
-        PushJoinThroughUnionRule.LEFT_UNION,
+    checkPlanning(JoinUnionTransposeRule.LEFT_UNION,
         "select r1.sal from "
-        + "(select * from emp e1 union all select * from emp e2) r1, "
-        + "emp r2");
+            + "(select * from emp e1 union all select * from emp e2) r1, "
+            + "emp r2");
   }
 
   @Test public void testPushJoinThroughUnionOnRight() {
-    checkPlanning(
-        PushJoinThroughUnionRule.RIGHT_UNION,
+    checkPlanning(JoinUnionTransposeRule.RIGHT_UNION,
         "select r1.sal from "
-        + "emp r1, "
-        + "(select * from emp e1 union all select * from emp e2) r2");
+            + "emp r1, "
+            + "(select * from emp e1 union all select * from emp e2) r2");
   }
 
   @Ignore("cycles")
   @Test public void testMergeFilterWithJoinCondition() throws Exception {
     HepProgram program = new HepProgramBuilder()
-        .addRuleInstance(TableAccessRule.INSTANCE)
-        .addRuleInstance(ExtractJoinFilterRule.INSTANCE)
+        .addRuleInstance(TableScanRule.INSTANCE)
+        .addRuleInstance(JoinExtractFilterRule.INSTANCE)
         .addRuleInstance(FilterToCalcRule.INSTANCE)
-        .addRuleInstance(MergeCalcRule.INSTANCE)
+        .addRuleInstance(CalcMergeRule.INSTANCE)
             //.addRuleInstance(FennelCalcRule.instance);
             //.addRuleInstance(FennelCartesianJoinRule.instance);
         .addRuleInstance(ProjectToCalcRule.INSTANCE)
@@ -388,9 +372,9 @@ public class RelOptRulesTest extends RelOptTestBase {
 
     checkPlanning(program,
         "select d.name as dname,e.ename as ename"
-        + " from emp e inner join dept d"
-        + " on e.deptno=d.deptno"
-        + " where d.name='Propane'");
+            + " from emp e inner join dept d"
+            + " on e.deptno=d.deptno"
+            + " where d.name='Propane'");
   }
 
   @Ignore("cycles")
@@ -406,7 +390,7 @@ public class RelOptRulesTest extends RelOptTestBase {
     // and verify that it only applies to one usage of the
     // table, not both (which would be incorrect).
     HepProgram program = new HepProgramBuilder()
-        .addRuleInstance(TableAccessRule.INSTANCE)
+        .addRuleInstance(TableScanRule.INSTANCE)
         .addRuleInstance(ProjectToCalcRule.INSTANCE)
 
             // Control the calc conversion.
@@ -418,53 +402,53 @@ public class RelOptRulesTest extends RelOptTestBase {
 
     checkPlanning(program,
         "select upper(ename) from emp union all"
-        + " select lower(ename) from emp");
+            + " select lower(ename) from emp");
   }
 
   @Test public void testPushSemiJoinPastJoinRuleLeft() throws Exception {
     // tests the case where the semijoin is pushed to the left
     HepProgram program = new HepProgramBuilder()
-        .addRuleInstance(PushFilterPastJoinRule.FILTER_ON_JOIN)
-        .addRuleInstance(AddRedundantSemiJoinRule.INSTANCE)
-        .addRuleInstance(PushSemiJoinPastJoinRule.INSTANCE)
+        .addRuleInstance(FilterJoinRule.FILTER_ON_JOIN)
+        .addRuleInstance(JoinAddRedundantSemiJoinRule.INSTANCE)
+        .addRuleInstance(SemiJoinJoinTransposeRule.INSTANCE)
         .build();
     checkPlanning(program,
         "select e1.ename from emp e1, dept d, emp e2 "
-        + "where e1.deptno = d.deptno and e1.empno = e2.empno");
+            + "where e1.deptno = d.deptno and e1.empno = e2.empno");
   }
 
   @Test public void testPushSemiJoinPastJoinRuleRight() throws Exception {
     // tests the case where the semijoin is pushed to the right
     HepProgram program = new HepProgramBuilder()
-        .addRuleInstance(PushFilterPastJoinRule.FILTER_ON_JOIN)
-        .addRuleInstance(AddRedundantSemiJoinRule.INSTANCE)
-        .addRuleInstance(PushSemiJoinPastJoinRule.INSTANCE)
+        .addRuleInstance(FilterJoinRule.FILTER_ON_JOIN)
+        .addRuleInstance(JoinAddRedundantSemiJoinRule.INSTANCE)
+        .addRuleInstance(SemiJoinJoinTransposeRule.INSTANCE)
         .build();
     checkPlanning(program,
         "select e1.ename from emp e1, dept d, emp e2 "
-        + "where e1.deptno = d.deptno and d.deptno = e2.deptno");
+            + "where e1.deptno = d.deptno and d.deptno = e2.deptno");
   }
 
   @Test public void testPushSemiJoinPastFilter() throws Exception {
     HepProgram program = new HepProgramBuilder()
-        .addRuleInstance(PushFilterPastJoinRule.FILTER_ON_JOIN)
-        .addRuleInstance(AddRedundantSemiJoinRule.INSTANCE)
-        .addRuleInstance(PushSemiJoinPastFilterRule.INSTANCE)
+        .addRuleInstance(FilterJoinRule.FILTER_ON_JOIN)
+        .addRuleInstance(JoinAddRedundantSemiJoinRule.INSTANCE)
+        .addRuleInstance(SemiJoinFilterTransposeRule.INSTANCE)
         .build();
     checkPlanning(program,
         "select e.ename from emp e, dept d "
-        + "where e.deptno = d.deptno and e.ename = 'foo'");
+            + "where e.deptno = d.deptno and e.ename = 'foo'");
   }
 
   @Test public void testConvertMultiJoinRule() throws Exception {
     HepProgram program = new HepProgramBuilder()
-        .addRuleInstance(PushFilterPastJoinRule.FILTER_ON_JOIN)
+        .addRuleInstance(FilterJoinRule.FILTER_ON_JOIN)
         .addMatchOrder(HepMatchOrder.BOTTOM_UP)
-        .addRuleInstance(ConvertMultiJoinRule.INSTANCE)
+        .addRuleInstance(JoinToMultiJoinRule.INSTANCE)
         .build();
     checkPlanning(program,
         "select e1.ename from emp e1, dept d, emp e2 "
-        + "where e1.deptno = d.deptno and d.deptno = e2.deptno");
+            + "where e1.deptno = d.deptno and d.deptno = e2.deptno");
   }
 
   @Test public void testReduceConstants() throws Exception {
@@ -482,10 +466,10 @@ public class RelOptRulesTest extends RelOptTestBase {
     // NULL" and we need "INTEGER".
     checkPlanning(program,
         "select 1+2, d.deptno+(3+4), (5+6)+d.deptno, cast(null as integer),"
-        + " coalesce(2,null), row(7+8)"
-        + " from dept d inner join emp e"
-        + " on d.deptno = e.deptno + (5-5)"
-        + " where d.deptno=(7+8) and d.deptno=coalesce(2,null)");
+            + " coalesce(2,null), row(7+8)"
+            + " from dept d inner join emp e"
+            + " on d.deptno = e.deptno + (5-5)"
+            + " where d.deptno=(7+8) and d.deptno=coalesce(2,null)");
   }
 
   @Test public void testReduceConstants2() throws Exception {
@@ -528,17 +512,17 @@ public class RelOptRulesTest extends RelOptTestBase {
     // and reduce it to TRUE. Only in the Calc are projects and conditions
     // combined.
     HepProgram program = new HepProgramBuilder()
-        .addRuleInstance(PushFilterPastProjectRule.INSTANCE)
-        .addRuleInstance(PushFilterPastSetOpRule.INSTANCE)
+        .addRuleInstance(FilterProjectTransposeRule.INSTANCE)
+        .addRuleInstance(FilterSetOpTransposeRule.INSTANCE)
         .addRuleInstance(FilterToCalcRule.INSTANCE)
         .addRuleInstance(ProjectToCalcRule.INSTANCE)
-        .addRuleInstance(MergeCalcRule.INSTANCE)
+        .addRuleInstance(CalcMergeRule.INSTANCE)
         .addRuleInstance(ReduceExpressionsRule.CALC_INSTANCE)
 
             // the hard part is done... a few more rule calls to clean up
-        .addRuleInstance(RemoveEmptyRules.UNION_INSTANCE)
+        .addRuleInstance(EmptyPruneRules.UNION_INSTANCE)
         .addRuleInstance(ProjectToCalcRule.INSTANCE)
-        .addRuleInstance(MergeCalcRule.INSTANCE)
+        .addRuleInstance(CalcMergeRule.INSTANCE)
         .addRuleInstance(ReduceExpressionsRule.CALC_INSTANCE)
         .build();
 
@@ -546,65 +530,65 @@ public class RelOptRulesTest extends RelOptTestBase {
     //  SELECT * FROM (VALUES ('TABLE        ', 'T')) AS T(U, S)
     checkPlanning(program,
         "select * from (\n"
-        + "  select upper(substring(x FROM 1 FOR 2) || substring(x FROM 3)) as u,\n"
-        + "      substring(x FROM 1 FOR 1) as s\n"
-        + "  from (\n"
-        + "    select 'table' as x from (values (true))\n"
-        + "    union\n"
-        + "    select 'view' from (values (true))\n"
-        + "    union\n"
-        + "    select 'foreign table' from (values (true))\n"
-        + "  )\n"
-        + ") where u = 'TABLE'");
+            + "  select upper(substring(x FROM 1 FOR 2) || substring(x FROM 3)) as u,\n"
+            + "      substring(x FROM 1 FOR 1) as s\n"
+            + "  from (\n"
+            + "    select 'table' as x from (values (true))\n"
+            + "    union\n"
+            + "    select 'view' from (values (true))\n"
+            + "    union\n"
+            + "    select 'foreign table' from (values (true))\n"
+            + "  )\n"
+            + ") where u = 'TABLE'");
   }
 
   @Test public void testRemoveSemiJoin() throws Exception {
     HepProgram program = new HepProgramBuilder()
-        .addRuleInstance(PushFilterPastJoinRule.FILTER_ON_JOIN)
-        .addRuleInstance(AddRedundantSemiJoinRule.INSTANCE)
-        .addRuleInstance(RemoveSemiJoinRule.INSTANCE)
+        .addRuleInstance(FilterJoinRule.FILTER_ON_JOIN)
+        .addRuleInstance(JoinAddRedundantSemiJoinRule.INSTANCE)
+        .addRuleInstance(SemiJoinRemoveRule.INSTANCE)
         .build();
     checkPlanning(program,
         "select e.ename from emp e, dept d "
-        + "where e.deptno = d.deptno");
+            + "where e.deptno = d.deptno");
   }
 
   @Test public void testRemoveSemiJoinWithFilter() throws Exception {
     HepProgram program = new HepProgramBuilder()
-        .addRuleInstance(PushFilterPastJoinRule.FILTER_ON_JOIN)
-        .addRuleInstance(AddRedundantSemiJoinRule.INSTANCE)
-        .addRuleInstance(PushSemiJoinPastFilterRule.INSTANCE)
-        .addRuleInstance(RemoveSemiJoinRule.INSTANCE)
+        .addRuleInstance(FilterJoinRule.FILTER_ON_JOIN)
+        .addRuleInstance(JoinAddRedundantSemiJoinRule.INSTANCE)
+        .addRuleInstance(SemiJoinFilterTransposeRule.INSTANCE)
+        .addRuleInstance(SemiJoinRemoveRule.INSTANCE)
         .build();
     checkPlanning(program,
         "select e.ename from emp e, dept d "
-        + "where e.deptno = d.deptno and e.ename = 'foo'");
+            + "where e.deptno = d.deptno and e.ename = 'foo'");
   }
 
   @Test public void testRemoveSemiJoinRight() throws Exception {
     HepProgram program = new HepProgramBuilder()
-        .addRuleInstance(PushFilterPastJoinRule.FILTER_ON_JOIN)
-        .addRuleInstance(AddRedundantSemiJoinRule.INSTANCE)
-        .addRuleInstance(PushSemiJoinPastJoinRule.INSTANCE)
-        .addRuleInstance(RemoveSemiJoinRule.INSTANCE)
+        .addRuleInstance(FilterJoinRule.FILTER_ON_JOIN)
+        .addRuleInstance(JoinAddRedundantSemiJoinRule.INSTANCE)
+        .addRuleInstance(SemiJoinJoinTransposeRule.INSTANCE)
+        .addRuleInstance(SemiJoinRemoveRule.INSTANCE)
         .build();
     checkPlanning(program,
         "select e1.ename from emp e1, dept d, emp e2 "
-        + "where e1.deptno = d.deptno and d.deptno = e2.deptno");
+            + "where e1.deptno = d.deptno and d.deptno = e2.deptno");
   }
 
   @Test public void testRemoveSemiJoinRightWithFilter() throws Exception {
     HepProgram program = new HepProgramBuilder()
-        .addRuleInstance(PushFilterPastJoinRule.FILTER_ON_JOIN)
-        .addRuleInstance(AddRedundantSemiJoinRule.INSTANCE)
-        .addRuleInstance(PushSemiJoinPastJoinRule.INSTANCE)
-        .addRuleInstance(PushSemiJoinPastFilterRule.INSTANCE)
-        .addRuleInstance(RemoveSemiJoinRule.INSTANCE)
+        .addRuleInstance(FilterJoinRule.FILTER_ON_JOIN)
+        .addRuleInstance(JoinAddRedundantSemiJoinRule.INSTANCE)
+        .addRuleInstance(SemiJoinJoinTransposeRule.INSTANCE)
+        .addRuleInstance(SemiJoinFilterTransposeRule.INSTANCE)
+        .addRuleInstance(SemiJoinRemoveRule.INSTANCE)
         .build();
     checkPlanning(program,
         "select e1.ename from emp e1, dept d, emp e2 "
-        + "where e1.deptno = d.deptno and d.deptno = e2.deptno "
-        + "and d.name = 'foo'");
+            + "where e1.deptno = d.deptno and d.deptno = e2.deptno "
+            + "and d.name = 'foo'");
   }
 
   private void checkPlanning(String query) throws Exception {
@@ -612,8 +596,7 @@ public class RelOptRulesTest extends RelOptTestBase {
         new Function<RelDataTypeFactory, Prepare.CatalogReader>() {
           public Prepare.CatalogReader apply(RelDataTypeFactory typeFactory) {
             return new MockCatalogReader(typeFactory, true) {
-              @Override
-              public MockCatalogReader init() {
+              @Override public MockCatalogReader init() {
                 // CREATE SCHEMA abc;
                 // CREATE TABLE a(a INT);
                 // ...
@@ -636,8 +619,8 @@ public class RelOptRulesTest extends RelOptTestBase {
         });
     HepProgram program = new HepProgramBuilder()
         .addMatchOrder(HepMatchOrder.BOTTOM_UP)
-        .addRuleInstance(RemoveTrivialProjectRule.INSTANCE)
-        .addRuleInstance(ConvertMultiJoinRule.INSTANCE)
+        .addRuleInstance(ProjectRemoveRule.INSTANCE)
+        .addRuleInstance(JoinToMultiJoinRule.INSTANCE)
         .build();
     checkPlanning(tester1, null,
         new HepPlanner(program), query);
@@ -665,38 +648,38 @@ public class RelOptRulesTest extends RelOptTestBase {
 
   @Test public void testConvertMultiJoinRuleOuterJoins2() throws Exception {
     // in (A right join B) join C, pushing C is not allowed;
-    // therefore there should be 2 MultiJoinRel
+    // therefore there should be 2 MultiJoin
     checkPlanning("select * from A right join B on a = b join C on b = c");
   }
 
   @Test public void testConvertMultiJoinRuleOuterJoins3() throws Exception {
     // in (A join B) left join C, pushing C is allowed;
-    // therefore there should be 1 MultiJoinRel
+    // therefore there should be 1 MultiJoin
     checkPlanning("select * from A join B on a = b left join C on b = c");
   }
 
   @Test public void testConvertMultiJoinRuleOuterJoins4() throws Exception {
     // in (A join B) right join C, pushing C is not allowed;
-    // therefore there should be 2 MultiJoinRel
+    // therefore there should be 2 MultiJoin
     checkPlanning("select * from A join B on a = b right join C on b = c");
   }
 
   @Test public void testPushSemiJoinPastProject() throws Exception {
     HepProgram program = new HepProgramBuilder()
-        .addRuleInstance(PushFilterPastJoinRule.FILTER_ON_JOIN)
-        .addRuleInstance(AddRedundantSemiJoinRule.INSTANCE)
-        .addRuleInstance(PushSemiJoinPastProjectRule.INSTANCE)
+        .addRuleInstance(FilterJoinRule.FILTER_ON_JOIN)
+        .addRuleInstance(JoinAddRedundantSemiJoinRule.INSTANCE)
+        .addRuleInstance(SemiJoinProjectTransposeRule.INSTANCE)
         .build();
     checkPlanning(program,
         "select e.* from "
-        + "(select ename, trim(job), sal * 2, deptno from emp) e, dept d "
-        + "where e.deptno = d.deptno");
+            + "(select ename, trim(job), sal * 2, deptno from emp) e, dept d "
+            + "where e.deptno = d.deptno");
   }
 
   @Test public void testReduceValuesUnderFilter() throws Exception {
     HepProgram program = new HepProgramBuilder()
-        .addRuleInstance(PushFilterPastProjectRule.INSTANCE)
-        .addRuleInstance(ReduceValuesRule.FILTER_INSTANCE)
+        .addRuleInstance(FilterProjectTransposeRule.INSTANCE)
+        .addRuleInstance(ValuesReduceRule.FILTER_INSTANCE)
         .build();
 
     // Plan should be same as for
@@ -707,8 +690,8 @@ public class RelOptRulesTest extends RelOptTestBase {
 
   @Test public void testReduceValuesUnderProject() throws Exception {
     HepProgram program = new HepProgramBuilder()
-        .addRuleInstance(MergeProjectRule.INSTANCE)
-        .addRuleInstance(ReduceValuesRule.PROJECT_INSTANCE)
+        .addRuleInstance(ProjectMergeRule.INSTANCE)
+        .addRuleInstance(ValuesReduceRule.PROJECT_INSTANCE)
         .build();
 
     // Plan should be same as for
@@ -719,16 +702,16 @@ public class RelOptRulesTest extends RelOptTestBase {
 
   @Test public void testReduceValuesUnderProjectFilter() throws Exception {
     HepProgram program = new HepProgramBuilder()
-        .addRuleInstance(PushFilterPastProjectRule.INSTANCE)
-        .addRuleInstance(MergeProjectRule.INSTANCE)
-        .addRuleInstance(ReduceValuesRule.PROJECT_FILTER_INSTANCE)
+        .addRuleInstance(FilterProjectTransposeRule.INSTANCE)
+        .addRuleInstance(ProjectMergeRule.INSTANCE)
+        .addRuleInstance(ValuesReduceRule.PROJECT_FILTER_INSTANCE)
         .build();
 
     // Plan should be same as for
     // select * from (values (11, 1, 10), (23, 3, 20)) as t(x, b, a)");
     checkPlanning(program,
         "select a + b as x, b, a from (values (10, 1), (30, 7), (20, 3)) as t(a, b)"
-        + " where a - b < 21");
+            + " where a - b < 21");
   }
 
   @Ignore // Calcite does not support INSERT yet
@@ -736,7 +719,7 @@ public class RelOptRulesTest extends RelOptTestBase {
     // The NULL literal presents pitfalls for value-reduction. Only
     // an INSERT statement contains un-CASTed NULL values.
     HepProgram program = new HepProgramBuilder()
-        .addRuleInstance(ReduceValuesRule.PROJECT_INSTANCE)
+        .addRuleInstance(ValuesReduceRule.PROJECT_INSTANCE)
         .build();
     checkPlanning(program,
         "insert into sales.depts(deptno,name) values (NULL, 'null')");
@@ -744,89 +727,89 @@ public class RelOptRulesTest extends RelOptTestBase {
 
   @Test public void testReduceValuesToEmpty() throws Exception {
     HepProgram program = new HepProgramBuilder()
-        .addRuleInstance(PushFilterPastProjectRule.INSTANCE)
-        .addRuleInstance(MergeProjectRule.INSTANCE)
-        .addRuleInstance(ReduceValuesRule.PROJECT_FILTER_INSTANCE)
+        .addRuleInstance(FilterProjectTransposeRule.INSTANCE)
+        .addRuleInstance(ProjectMergeRule.INSTANCE)
+        .addRuleInstance(ValuesReduceRule.PROJECT_FILTER_INSTANCE)
         .build();
 
     // Plan should be same as for
     // select * from (values (11, 1, 10), (23, 3, 20)) as t(x, b, a)");
     checkPlanning(program,
         "select a + b as x, b, a from (values (10, 1), (30, 7)) as t(a, b)"
-        + " where a - b < 0");
+            + " where a - b < 0");
   }
 
   @Test public void testEmptyFilterProjectUnion() throws Exception {
     HepProgram program = new HepProgramBuilder()
-        .addRuleInstance(PushFilterPastSetOpRule.INSTANCE)
-        .addRuleInstance(PushFilterPastProjectRule.INSTANCE)
-        .addRuleInstance(MergeProjectRule.INSTANCE)
-        .addRuleInstance(ReduceValuesRule.PROJECT_FILTER_INSTANCE)
-        .addRuleInstance(RemoveEmptyRules.PROJECT_INSTANCE)
-        .addRuleInstance(RemoveEmptyRules.UNION_INSTANCE)
+        .addRuleInstance(FilterSetOpTransposeRule.INSTANCE)
+        .addRuleInstance(FilterProjectTransposeRule.INSTANCE)
+        .addRuleInstance(ProjectMergeRule.INSTANCE)
+        .addRuleInstance(ValuesReduceRule.PROJECT_FILTER_INSTANCE)
+        .addRuleInstance(EmptyPruneRules.PROJECT_INSTANCE)
+        .addRuleInstance(EmptyPruneRules.UNION_INSTANCE)
         .build();
 
     // Plan should be same as for
     // select * from (values (30, 3)) as t(x, y)");
     checkPlanning(program,
         "select * from (\n"
-        + "select * from (values (10, 1), (30, 3)) as t (x, y)\n"
-        + "union all\n"
-        + "select * from (values (20, 2))\n"
-        + ")\n"
-        + "where x + y > 30");
+            + "select * from (values (10, 1), (30, 3)) as t (x, y)\n"
+            + "union all\n"
+            + "select * from (values (20, 2))\n"
+            + ")\n"
+            + "where x + y > 30");
   }
 
   @Test public void testEmptyJoin() {
     HepProgram program = new HepProgramBuilder()
         .addRuleInstance(ReduceExpressionsRule.FILTER_INSTANCE)
-        .addRuleInstance(RemoveEmptyRules.PROJECT_INSTANCE)
-        .addRuleInstance(RemoveEmptyRules.JOIN_LEFT_INSTANCE)
-        .addRuleInstance(RemoveEmptyRules.JOIN_RIGHT_INSTANCE)
+        .addRuleInstance(EmptyPruneRules.PROJECT_INSTANCE)
+        .addRuleInstance(EmptyPruneRules.JOIN_LEFT_INSTANCE)
+        .addRuleInstance(EmptyPruneRules.JOIN_RIGHT_INSTANCE)
         .build();
 
     // Plan should be empty
     checkPlanning(program,
         "select * from (\n"
-        + "select * from emp where false)\n"
-        + "join dept using (deptno)");
+            + "select * from emp where false)\n"
+            + "join dept using (deptno)");
   }
 
   @Test public void testEmptyJoinLeft() {
     HepProgram program = new HepProgramBuilder()
         .addRuleInstance(ReduceExpressionsRule.FILTER_INSTANCE)
-        .addRuleInstance(RemoveEmptyRules.PROJECT_INSTANCE)
-        .addRuleInstance(RemoveEmptyRules.JOIN_LEFT_INSTANCE)
-        .addRuleInstance(RemoveEmptyRules.JOIN_RIGHT_INSTANCE)
+        .addRuleInstance(EmptyPruneRules.PROJECT_INSTANCE)
+        .addRuleInstance(EmptyPruneRules.JOIN_LEFT_INSTANCE)
+        .addRuleInstance(EmptyPruneRules.JOIN_RIGHT_INSTANCE)
         .build();
 
     // Plan should be empty
     checkPlanning(program,
         "select * from (\n"
-        + "select * from emp where false)\n"
-        + "left join dept using (deptno)");
+            + "select * from emp where false)\n"
+            + "left join dept using (deptno)");
   }
 
   @Test public void testEmptyJoinRight() {
     HepProgram program = new HepProgramBuilder()
         .addRuleInstance(ReduceExpressionsRule.FILTER_INSTANCE)
-        .addRuleInstance(RemoveEmptyRules.PROJECT_INSTANCE)
-        .addRuleInstance(RemoveEmptyRules.JOIN_LEFT_INSTANCE)
-        .addRuleInstance(RemoveEmptyRules.JOIN_RIGHT_INSTANCE)
+        .addRuleInstance(EmptyPruneRules.PROJECT_INSTANCE)
+        .addRuleInstance(EmptyPruneRules.JOIN_LEFT_INSTANCE)
+        .addRuleInstance(EmptyPruneRules.JOIN_RIGHT_INSTANCE)
         .build();
 
     // Plan should be equivalent to "select * from emp join dept".
     // Cannot optimize away the join because of RIGHT.
     checkPlanning(program,
         "select * from (\n"
-        + "select * from emp where false)\n"
-        + "right join dept using (deptno)");
+            + "select * from emp where false)\n"
+            + "right join dept using (deptno)");
   }
 
   @Test public void testEmptySort() {
     HepProgram program = new HepProgramBuilder()
         .addRuleInstance(ReduceExpressionsRule.FILTER_INSTANCE)
-        .addRuleInstance(RemoveEmptyRules.SORT_INSTANCE)
+        .addRuleInstance(EmptyPruneRules.SORT_INSTANCE)
         .build();
 
     checkPlanning(program,
@@ -835,7 +818,7 @@ public class RelOptRulesTest extends RelOptTestBase {
 
   @Test public void testEmptySortLimitZero() {
     HepProgram program = new HepProgramBuilder()
-        .addRuleInstance(RemoveEmptyRules.SORT_FETCH_ZERO_INSTANCE)
+        .addRuleInstance(EmptyPruneRules.SORT_FETCH_ZERO_INSTANCE)
         .build();
 
     checkPlanning(program,
@@ -852,9 +835,9 @@ public class RelOptRulesTest extends RelOptTestBase {
     // The resulting plan should have no cast expressions
     checkPlanning(program,
         "select cast(d.name as varchar(128)), cast(e.empno as integer) "
-        + "from dept as d inner join emp as e "
-        + "on cast(d.deptno as integer) = cast(e.deptno as integer) "
-        + "where cast(e.job as varchar(1)) = 'Manager'");
+            + "from dept as d inner join emp as e "
+            + "on cast(d.deptno as integer) = cast(e.deptno as integer) "
+            + "where cast(e.job as varchar(1)) = 'Manager'");
   }
 
   @Test public void testReduceCastAndConsts() throws Exception {
@@ -866,7 +849,7 @@ public class RelOptRulesTest extends RelOptTestBase {
     // in addition to the casts.
     checkPlanning(program,
         "select * from emp "
-        + "where cast((empno + (10/2)) as int) = 13");
+            + "where cast((empno + (10/2)) as int) = 13");
   }
 
   @Ignore // Calcite does not support INSERT yet
@@ -875,24 +858,24 @@ public class RelOptRulesTest extends RelOptTestBase {
 
         // Simulate the way INSERT will insert casts to the target types
         .addRuleInstance(
-            new CoerceInputsRule(TableModificationRel.class, false))
+            new CoerceInputsRule(LogicalTableModify.class, false))
 
             // Convert projects to calcs, merge two calcs, and then
             // reduce redundant casts in merged calc.
         .addRuleInstance(ProjectToCalcRule.INSTANCE)
-        .addRuleInstance(MergeCalcRule.INSTANCE)
+        .addRuleInstance(CalcMergeRule.INSTANCE)
         .addRuleInstance(ReduceExpressionsRule.CALC_INSTANCE)
         .build();
     checkPlanning(program,
         "insert into sales.depts(name) "
-        + "select cast(gender as varchar(128)) from sales.emps");
+            + "select cast(gender as varchar(128)) from sales.emps");
   }
 
   private void basePushAggThroughUnion() throws Exception {
     HepProgram program = new HepProgramBuilder()
-        .addRuleInstance(PushProjectPastSetOpRule.INSTANCE)
-        .addRuleInstance(MergeProjectRule.INSTANCE)
-        .addRuleInstance(PushAggregateThroughUnionRule.INSTANCE)
+        .addRuleInstance(ProjectSetOpTransposeRule.INSTANCE)
+        .addRuleInstance(ProjectMergeRule.INSTANCE)
+        .addRuleInstance(AggregateUnionTransposeRule.INSTANCE)
         .build();
     checkPlanning(program, "${sql}");
   }
@@ -940,9 +923,9 @@ public class RelOptRulesTest extends RelOptTestBase {
 
   private void basePullConstantTroughAggregate() throws Exception {
     HepProgram program = new HepProgramBuilder()
-        .addRuleInstance(MergeProjectRule.INSTANCE)
-        .addRuleInstance(PullConstantsThroughAggregatesRule.INSTANCE)
-        .addRuleInstance(MergeProjectRule.INSTANCE)
+        .addRuleInstance(ProjectMergeRule.INSTANCE)
+        .addRuleInstance(AggregateProjectPullUpConstantsRule.INSTANCE)
+        .addRuleInstance(ProjectMergeRule.INSTANCE)
         .build();
     checkPlanning(program, "${sql}");
   }
@@ -993,9 +976,9 @@ public class RelOptRulesTest extends RelOptTestBase {
         .build();
     checkPlanning(program,
         "select x, sum(z), y from (\n"
-        + "  select deptno as x, empno as y, sal as z, sal * 2 as zz\n"
-        + "  from emp)\n"
-        + "group by x, y");
+            + "  select deptno as x, empno as y, sal as z, sal * 2 as zz\n"
+            + "  from emp)\n"
+            + "group by x, y");
   }
 
   public void transitiveInference() throws Exception {
@@ -1003,10 +986,10 @@ public class RelOptRulesTest extends RelOptTestBase {
     String sql = diffRepos.expand(null, "${sql}");
 
     HepProgram program = new HepProgramBuilder()
-        .addRuleInstance(PushFilterPastJoinRule.DUMB_FILTER_ON_JOIN)
-        .addRuleInstance(PushFilterPastJoinRule.JOIN)
-        .addRuleInstance(PushFilterPastProjectRule.INSTANCE)
-        .addRuleInstance(PushFilterPastSetOpRule.INSTANCE)
+        .addRuleInstance(FilterJoinRule.DUMB_FILTER_ON_JOIN)
+        .addRuleInstance(FilterJoinRule.JOIN)
+        .addRuleInstance(FilterProjectTransposeRule.INSTANCE)
+        .addRuleInstance(FilterSetOpTransposeRule.INSTANCE)
         .build();
     HepPlanner planner = new HepPlanner(program);
 
@@ -1031,11 +1014,11 @@ public class RelOptRulesTest extends RelOptTestBase {
 
     HepProgram program2 = new HepProgramBuilder()
         .addMatchOrder(HepMatchOrder.BOTTOM_UP)
-        .addRuleInstance(PushFilterPastJoinRule.DUMB_FILTER_ON_JOIN)
-        .addRuleInstance(PushFilterPastJoinRule.JOIN)
-        .addRuleInstance(PushFilterPastProjectRule.INSTANCE)
-        .addRuleInstance(PushFilterPastSetOpRule.INSTANCE)
-        .addRuleInstance(TransitivePredicatesOnJoinRule.INSTANCE)
+        .addRuleInstance(FilterJoinRule.DUMB_FILTER_ON_JOIN)
+        .addRuleInstance(FilterJoinRule.JOIN)
+        .addRuleInstance(FilterProjectTransposeRule.INSTANCE)
+        .addRuleInstance(FilterSetOpTransposeRule.INSTANCE)
+        .addRuleInstance(JoinPushTransitivePredicatesRule.INSTANCE)
         .build();
     HepPlanner planner2 = new HepPlanner(program2);
     planner.registerMetadataProviders(list);
@@ -1126,7 +1109,7 @@ public class RelOptRulesTest extends RelOptTestBase {
 
   @Test public void testPushFilterWithRank() throws Exception {
     HepProgram program = new HepProgramBuilder().addRuleInstance(
-        PushFilterPastProjectRule.INSTANCE).build();
+        FilterProjectTransposeRule.INSTANCE).build();
     checkPlanning(program, "select e1.ename, r\n"
         + "from (\n"
         + "  select ename, "
@@ -1137,7 +1120,7 @@ public class RelOptRulesTest extends RelOptTestBase {
 
   @Test public void testPushFilterWithRankExpr() throws Exception {
     HepProgram program = new HepProgramBuilder().addRuleInstance(
-        PushFilterPastProjectRule.INSTANCE).build();
+        FilterProjectTransposeRule.INSTANCE).build();
     checkPlanning(program, "select e1.ename, r\n"
         + "from (\n"
         + "  select ename,\n"

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/test/RelOptTestBase.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/RelOptTestBase.java b/core/src/test/java/org/apache/calcite/test/RelOptTestBase.java
index faebe47..790aa25 100644
--- a/core/src/test/java/org/apache/calcite/test/RelOptTestBase.java
+++ b/core/src/test/java/org/apache/calcite/test/RelOptTestBase.java
@@ -14,18 +14,24 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.test;
-
-import java.util.List;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.rel.metadata.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.relopt.hep.*;
+package org.apache.calcite.test;
+
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.hep.HepPlanner;
+import org.apache.calcite.plan.hep.HepProgram;
+import org.apache.calcite.plan.hep.HepProgramBuilder;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.metadata.ChainedRelMetadataProvider;
+import org.apache.calcite.rel.metadata.DefaultRelMetadataProvider;
+import org.apache.calcite.rel.metadata.RelMetadataProvider;
 
 import com.google.common.collect.Lists;
 
-import static org.junit.Assert.*;
+import java.util.List;
+
+import static org.junit.Assert.assertTrue;
 
 /**
  * RelOptTestBase is an abstract base for tests which exercise a planner and/or
@@ -62,12 +68,8 @@ abstract class RelOptTestBase extends SqlToRelTestBase {
    * @param program Planner program
    * @param sql     SQL query
    */
-  protected void checkPlanning(
-      HepProgram program,
-      String sql) {
-    checkPlanning(
-        new HepPlanner(program),
-        sql);
+  protected void checkPlanning(HepProgram program, String sql) {
+    checkPlanning(new HepPlanner(program), sql);
   }
 
   /**
@@ -77,9 +79,7 @@ abstract class RelOptTestBase extends SqlToRelTestBase {
    * @param planner Planner
    * @param sql     SQL query
    */
-  protected void checkPlanning(
-      RelOptPlanner planner,
-      String sql) {
+  protected void checkPlanning(RelOptPlanner planner, String sql) {
     checkPlanning(tester, null, planner, sql);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/test/RexProgramTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/RexProgramTest.java b/core/src/test/java/org/apache/calcite/test/RexProgramTest.java
index b3d8ebb..d1d0bae 100644
--- a/core/src/test/java/org/apache/calcite/test/RexProgramTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RexProgramTest.java
@@ -14,22 +14,29 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.test;
-
-import java.math.BigDecimal;
-import java.util.*;
-
-import org.eigenbase.relopt.Strong;
-import org.eigenbase.reltype.*;
-import org.eigenbase.rex.*;
-import org.eigenbase.sql.fun.SqlStdOperatorTable;
-import org.eigenbase.sql.type.SqlTypeName;
-import org.eigenbase.util.*;
-
-import net.hydromatic.optiq.impl.java.JavaTypeFactory;
-import net.hydromatic.optiq.jdbc.JavaTypeFactoryImpl;
-import net.hydromatic.optiq.test.OptiqAssert;
-import net.hydromatic.optiq.util.BitSets;
+package org.apache.calcite.test;
+
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
+import org.apache.calcite.plan.Strong;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexDynamicParam;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexLocalRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexProgramBuilder;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.TestUtil;
+import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
@@ -37,13 +44,18 @@ import com.google.common.collect.Lists;
 import org.junit.Before;
 import org.junit.Test;
 
+import java.math.BigDecimal;
+import java.util.Arrays;
+import java.util.BitSet;
+import java.util.List;
+
 import static org.hamcrest.CoreMatchers.equalTo;
 import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertThat;
 
 /**
  * Unit tests for {@link RexProgram} and
- * {@link org.eigenbase.rex.RexProgramBuilder}.
+ * {@link org.apache.calcite.rex.RexProgramBuilder}.
  */
 public class RexProgramTest {
   //~ Instance fields --------------------------------------------------------
@@ -126,8 +138,8 @@ public class RexProgramTest {
     final String programString = program.toString();
     TestUtil.assertEqualsVerbose(
         "(expr#0..1=[{inputs}], expr#2=[+($0, 1)], expr#3=[77], "
-        + "expr#4=[+($0, $1)], expr#5=[+($0, $0)], expr#6=[+($t4, $t2)], "
-        + "a=[$t6], b=[$t5])",
+            + "expr#4=[+($0, $1)], expr#5=[+($0, $0)], expr#6=[+($t4, $t2)], "
+            + "a=[$t6], b=[$t5])",
         programString);
 
     // Normalize the program using the RexProgramBuilder.normalize API.
@@ -140,8 +152,8 @@ public class RexProgramTest {
     final String normalizedProgramString = normalizedProgram.toString();
     TestUtil.assertEqualsVerbose(
         "(expr#0..1=[{inputs}], expr#2=[+($t0, $t1)], expr#3=[1], "
-        + "expr#4=[+($t0, $t3)], expr#5=[+($t2, $t4)], "
-        + "expr#6=[+($t0, $t0)], a=[$t5], b=[$t6])",
+            + "expr#4=[+($t0, $t3)], expr#5=[+($t2, $t4)], "
+            + "expr#6=[+($t0, $t0)], a=[$t5], b=[$t6])",
         normalizedProgramString);
   }
 
@@ -166,8 +178,8 @@ public class RexProgramTest {
     final String unnormalizedProgram = builder.getProgram(false).toString();
     TestUtil.assertEqualsVerbose(
         "(expr#0..1=[{inputs}], expr#2=[+($0, 1)], expr#3=[77], "
-        + "expr#4=[+($0, $1)], expr#5=[+($0, 1)], expr#6=[+($0, $t5)], "
-        + "expr#7=[+($t4, $t2)], a=[$t7], b=[$t6])",
+            + "expr#4=[+($0, $1)], expr#5=[+($0, 1)], expr#6=[+($0, $t5)], "
+            + "expr#7=[+($t4, $t2)], a=[$t7], b=[$t6])",
         unnormalizedProgram);
 
     // normalize eliminates duplicates (specifically "+($0, $1)")
@@ -175,8 +187,8 @@ public class RexProgramTest {
     final String program2 = builder2.getProgram(true).toString();
     TestUtil.assertEqualsVerbose(
         "(expr#0..1=[{inputs}], expr#2=[+($t0, $t1)], expr#3=[1], "
-        + "expr#4=[+($t0, $t3)], expr#5=[+($t2, $t4)], "
-        + "expr#6=[+($t0, $t4)], a=[$t5], b=[$t6])",
+            + "expr#4=[+($t0, $t3)], expr#5=[+($t2, $t4)], "
+            + "expr#6=[+($t0, $t4)], a=[$t5], b=[$t6])",
         program2);
   }
 
@@ -191,10 +203,10 @@ public class RexProgramTest {
     final String program = builder.getProgram(true).toString();
     TestUtil.assertEqualsVerbose(
         "(expr#0..1=[{inputs}], expr#2=[+($t0, $t1)], expr#3=[1], "
-        + "expr#4=[+($t0, $t3)], expr#5=[+($t2, $t4)], "
-        + "expr#6=[+($t0, $t0)], expr#7=[>($t2, $t0)], "
-        + "expr#8=[AND($t7, $t7)], expr#9=[AND($t8, $t7)], "
-        + "a=[$t5], b=[$t6], $condition=[$t9])",
+            + "expr#4=[+($t0, $t3)], expr#5=[+($t2, $t4)], "
+            + "expr#6=[+($t0, $t0)], expr#7=[>($t2, $t0)], "
+            + "expr#8=[AND($t7, $t7)], expr#9=[AND($t8, $t7)], "
+            + "a=[$t5], b=[$t6], $condition=[$t9])",
         program);
   }
 
@@ -314,7 +326,7 @@ public class RexProgramTest {
     return Strong.is(e, b);
   }
 
-  /** Unit test for {@link org.eigenbase.relopt.Strong}. */
+  /** Unit test for {@link org.apache.calcite.plan.Strong}. */
   @Test public void testStrong() {
     final RelDataType intType = typeFactory.createSqlType(SqlTypeName.INTEGER);
 
@@ -362,7 +374,7 @@ public class RexProgramTest {
     assertThat(strongIf(andFalseTrue, c), is(false));
   }
 
-  /** Unit test for {@link org.eigenbase.rex.RexUtil#toCnf}. */
+  /** Unit test for {@link org.apache.calcite.rex.RexUtil#toCnf}. */
   @Test public void testCnf() {
     final RelDataType booleanType =
         typeFactory.createSqlType(SqlTypeName.BOOLEAN);
@@ -478,40 +490,40 @@ public class RexProgramTest {
                 eq(aRef, literal3),
                 eq(bRef, literal3))),
         "AND("
-        + "OR(=(?0.x, 1), =(?0.x, 2), =(?0.x, 3)), "
-        + "OR(=(?0.x, 1), =(?0.x, 2), =(?0.a, 3)), "
-        + "OR(=(?0.x, 1), =(?0.x, 2), =(?0.b, 3)), "
-        + "OR(=(?0.x, 1), =(?0.y, 2), =(?0.x, 3)), "
-        + "OR(=(?0.x, 1), =(?0.y, 2), =(?0.a, 3)), "
-        + "OR(=(?0.x, 1), =(?0.y, 2), =(?0.b, 3)), "
-        + "OR(=(?0.x, 1), =(?0.a, 2), =(?0.x, 3)), "
-        + "OR(=(?0.x, 1), =(?0.a, 2), =(?0.a, 3)), "
-        + "OR(=(?0.x, 1), =(?0.a, 2), =(?0.b, 3)), "
-        + "OR(=(?0.y, 1), =(?0.x, 2), =(?0.x, 3)), "
-        + "OR(=(?0.y, 1), =(?0.x, 2), =(?0.a, 3)), "
-        + "OR(=(?0.y, 1), =(?0.x, 2), =(?0.b, 3)), "
-        + "OR(=(?0.y, 1), =(?0.y, 2), =(?0.x, 3)), "
-        + "OR(=(?0.y, 1), =(?0.y, 2), =(?0.a, 3)), "
-        + "OR(=(?0.y, 1), =(?0.y, 2), =(?0.b, 3)), "
-        + "OR(=(?0.y, 1), =(?0.a, 2), =(?0.x, 3)), "
-        + "OR(=(?0.y, 1), =(?0.a, 2), =(?0.a, 3)), "
-        + "OR(=(?0.y, 1), =(?0.a, 2), =(?0.b, 3)), "
-        + "OR(=(?0.z, 1), =(?0.x, 2), =(?0.x, 3)), "
-        + "OR(=(?0.z, 1), =(?0.x, 2), =(?0.a, 3)), "
-        + "OR(=(?0.z, 1), =(?0.x, 2), =(?0.b, 3)), "
-        + "OR(=(?0.z, 1), =(?0.y, 2), =(?0.x, 3)), "
-        + "OR(=(?0.z, 1), =(?0.y, 2), =(?0.a, 3)), "
-        + "OR(=(?0.z, 1), =(?0.y, 2), =(?0.b, 3)), "
-        + "OR(=(?0.z, 1), =(?0.a, 2), =(?0.x, 3)), "
-        + "OR(=(?0.z, 1), =(?0.a, 2), =(?0.a, 3)), "
-        + "OR(=(?0.z, 1), =(?0.a, 2), =(?0.b, 3)))");
+            + "OR(=(?0.x, 1), =(?0.x, 2), =(?0.x, 3)), "
+            + "OR(=(?0.x, 1), =(?0.x, 2), =(?0.a, 3)), "
+            + "OR(=(?0.x, 1), =(?0.x, 2), =(?0.b, 3)), "
+            + "OR(=(?0.x, 1), =(?0.y, 2), =(?0.x, 3)), "
+            + "OR(=(?0.x, 1), =(?0.y, 2), =(?0.a, 3)), "
+            + "OR(=(?0.x, 1), =(?0.y, 2), =(?0.b, 3)), "
+            + "OR(=(?0.x, 1), =(?0.a, 2), =(?0.x, 3)), "
+            + "OR(=(?0.x, 1), =(?0.a, 2), =(?0.a, 3)), "
+            + "OR(=(?0.x, 1), =(?0.a, 2), =(?0.b, 3)), "
+            + "OR(=(?0.y, 1), =(?0.x, 2), =(?0.x, 3)), "
+            + "OR(=(?0.y, 1), =(?0.x, 2), =(?0.a, 3)), "
+            + "OR(=(?0.y, 1), =(?0.x, 2), =(?0.b, 3)), "
+            + "OR(=(?0.y, 1), =(?0.y, 2), =(?0.x, 3)), "
+            + "OR(=(?0.y, 1), =(?0.y, 2), =(?0.a, 3)), "
+            + "OR(=(?0.y, 1), =(?0.y, 2), =(?0.b, 3)), "
+            + "OR(=(?0.y, 1), =(?0.a, 2), =(?0.x, 3)), "
+            + "OR(=(?0.y, 1), =(?0.a, 2), =(?0.a, 3)), "
+            + "OR(=(?0.y, 1), =(?0.a, 2), =(?0.b, 3)), "
+            + "OR(=(?0.z, 1), =(?0.x, 2), =(?0.x, 3)), "
+            + "OR(=(?0.z, 1), =(?0.x, 2), =(?0.a, 3)), "
+            + "OR(=(?0.z, 1), =(?0.x, 2), =(?0.b, 3)), "
+            + "OR(=(?0.z, 1), =(?0.y, 2), =(?0.x, 3)), "
+            + "OR(=(?0.z, 1), =(?0.y, 2), =(?0.a, 3)), "
+            + "OR(=(?0.z, 1), =(?0.y, 2), =(?0.b, 3)), "
+            + "OR(=(?0.z, 1), =(?0.a, 2), =(?0.x, 3)), "
+            + "OR(=(?0.z, 1), =(?0.a, 2), =(?0.a, 3)), "
+            + "OR(=(?0.z, 1), =(?0.a, 2), =(?0.b, 3)))");
   }
 
   /** Tests formulas of various sizes whose size is exponential when converted
    * to CNF. */
   @Test public void testCnfExponential() {
     // run out of memory if limit is higher than about 20
-    final int limit = OptiqAssert.ENABLE_SLOW ? 16 : 6;
+    final int limit = CalciteAssert.ENABLE_SLOW ? 16 : 6;
     for (int i = 2; i < limit; i++) {
       checkExponentialCnf(i);
     }
@@ -538,15 +550,14 @@ public class RexProgramTest {
     assertThat((n + 1) * (int) Math.pow(2, n) + 1, equalTo(nodeCount));
     if (n == 3) {
       assertThat(cnf.toString(),
-          equalTo(
-              "AND(OR(?0.x0, ?0.x1, ?0.x2), OR(?0.x0, ?0.x1, ?0.y2),"
+          equalTo("AND(OR(?0.x0, ?0.x1, ?0.x2), OR(?0.x0, ?0.x1, ?0.y2),"
               + " OR(?0.x0, ?0.y1, ?0.x2), OR(?0.x0, ?0.y1, ?0.y2),"
               + " OR(?0.y0, ?0.x1, ?0.x2), OR(?0.y0, ?0.x1, ?0.y2),"
               + " OR(?0.y0, ?0.y1, ?0.x2), OR(?0.y0, ?0.y1, ?0.y2))"));
     }
   }
 
-  /** Unit test for {@link org.eigenbase.rex.RexUtil#pullFactors}. */
+  /** Unit test for {@link org.apache.calcite.rex.RexUtil#pullFactors}. */
   @Test public void testPullFactors() {
     final RelDataType booleanType =
         typeFactory.createSqlType(SqlTypeName.BOOLEAN);

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/test/RexTransformerTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/RexTransformerTest.java b/core/src/test/java/org/apache/calcite/test/RexTransformerTest.java
index 95f0a58..8f8f799 100644
--- a/core/src/test/java/org/apache/calcite/test/RexTransformerTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RexTransformerTest.java
@@ -14,19 +14,25 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.test;
+package org.apache.calcite.test;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.rex.*;
-import org.eigenbase.sql.fun.*;
-import org.eigenbase.sql.type.*;
-
-import net.hydromatic.optiq.jdbc.JavaTypeFactoryImpl;
+import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexTransformer;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.type.SqlTypeName;
 
 import org.junit.Before;
 import org.junit.Test;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 /**
  * Tests transformations on rex nodes.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/test/ScannableTableTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/ScannableTableTest.java b/core/src/test/java/org/apache/calcite/test/ScannableTableTest.java
index ed989ab..78bdab7 100644
--- a/core/src/test/java/org/apache/calcite/test/ScannableTableTest.java
+++ b/core/src/test/java/org/apache/calcite/test/ScannableTableTest.java
@@ -14,33 +14,42 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.test;
-
-import net.hydromatic.linq4j.AbstractEnumerable;
-import net.hydromatic.linq4j.Enumerable;
-import net.hydromatic.linq4j.Enumerator;
-
-import net.hydromatic.optiq.*;
-import net.hydromatic.optiq.impl.AbstractSchema;
-import net.hydromatic.optiq.impl.AbstractTable;
-import net.hydromatic.optiq.impl.java.ReflectiveSchema;
-import net.hydromatic.optiq.jdbc.OptiqConnection;
-
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.reltype.RelDataTypeFactory;
-import org.eigenbase.rex.RexCall;
-import org.eigenbase.rex.RexInputRef;
-import org.eigenbase.rex.RexLiteral;
-import org.eigenbase.rex.RexNode;
-import org.eigenbase.sql.fun.SqlStdOperatorTable;
-import org.eigenbase.sql.type.SqlTypeName;
-import org.eigenbase.util.Bug;
+package org.apache.calcite.test;
+
+import org.apache.calcite.DataContext;
+import org.apache.calcite.adapter.java.ReflectiveSchema;
+import org.apache.calcite.jdbc.CalciteConnection;
+import org.apache.calcite.linq4j.AbstractEnumerable;
+import org.apache.calcite.linq4j.Enumerable;
+import org.apache.calcite.linq4j.Enumerator;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.schema.FilterableTable;
+import org.apache.calcite.schema.ProjectableFilterableTable;
+import org.apache.calcite.schema.ScannableTable;
+import org.apache.calcite.schema.Schema;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.schema.Statistic;
+import org.apache.calcite.schema.Statistics;
+import org.apache.calcite.schema.impl.AbstractSchema;
+import org.apache.calcite.schema.impl.AbstractTable;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.util.Bug;
 
 import org.junit.Assert;
 import org.junit.Test;
 
 import java.math.BigDecimal;
-import java.sql.*;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
 import java.util.Arrays;
 import java.util.Iterator;
 import java.util.List;
@@ -51,7 +60,7 @@ import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 
 /**
- * Unit test for {@link net.hydromatic.optiq.ScannableTable}.
+ * Unit test for {@link org.apache.calcite.schema.ScannableTable}.
  */
 public class ScannableTableTest {
   @Test public void testTens() throws SQLException {
@@ -72,15 +81,15 @@ public class ScannableTableTest {
   @Test public void testSimple() throws Exception {
     Connection connection =
         DriverManager.getConnection("jdbc:calcite:");
-    OptiqConnection optiqConnection =
-        connection.unwrap(OptiqConnection.class);
-    SchemaPlus rootSchema = optiqConnection.getRootSchema();
+    CalciteConnection calciteConnection =
+        connection.unwrap(CalciteConnection.class);
+    SchemaPlus rootSchema = calciteConnection.getRootSchema();
     SchemaPlus schema = rootSchema.add("s", new AbstractSchema());
     schema.add("simple", new SimpleTable());
     rootSchema.add("hr", new ReflectiveSchema(new JdbcTest.HrSchema()));
     ResultSet resultSet = connection.createStatement().executeQuery(
         "select * from \"s\".\"simple\"");
-    assertThat(OptiqAssert.toString(resultSet),
+    assertThat(CalciteAssert.toString(resultSet),
         equalTo("i=0\ni=10\ni=20\ni=30\n"));
   }
 
@@ -88,15 +97,15 @@ public class ScannableTableTest {
   @Test public void testSimple2() throws Exception {
     Connection connection =
         DriverManager.getConnection("jdbc:calcite:");
-    OptiqConnection optiqConnection =
-        connection.unwrap(OptiqConnection.class);
-    SchemaPlus rootSchema = optiqConnection.getRootSchema();
+    CalciteConnection calciteConnection =
+        connection.unwrap(CalciteConnection.class);
+    SchemaPlus rootSchema = calciteConnection.getRootSchema();
     SchemaPlus schema = rootSchema.add("s", new AbstractSchema());
     schema.add("beatles", new BeatlesTable());
     rootSchema.add("hr", new ReflectiveSchema(new JdbcTest.HrSchema()));
     ResultSet resultSet = connection.createStatement().executeQuery(
         "select * from \"s\".\"beatles\"");
-    assertThat(OptiqAssert.toString(resultSet),
+    assertThat(CalciteAssert.toString(resultSet),
         equalTo("i=4; j=John\ni=4; j=Paul\ni=6; j=George\ni=5; j=Ringo\n"));
   }
 
@@ -104,16 +113,16 @@ public class ScannableTableTest {
   @Test public void testSimpleFilter2() throws Exception {
     Connection connection =
         DriverManager.getConnection("jdbc:calcite:");
-    OptiqConnection optiqConnection =
-        connection.unwrap(OptiqConnection.class);
-    SchemaPlus rootSchema = optiqConnection.getRootSchema();
+    CalciteConnection calciteConnection =
+        connection.unwrap(CalciteConnection.class);
+    SchemaPlus rootSchema = calciteConnection.getRootSchema();
     SchemaPlus schema = rootSchema.add("s", new AbstractSchema());
     final StringBuilder buf = new StringBuilder();
     schema.add("beatles", new BeatlesFilterableTable(buf, true));
     final Statement statement = connection.createStatement();
     ResultSet resultSet = statement.executeQuery(
         "select * from \"s\".\"beatles\" where \"i\" = 4");
-    assertThat(OptiqAssert.toString(resultSet),
+    assertThat(CalciteAssert.toString(resultSet),
         equalTo("i=4; j=John; k=1940\ni=4; j=Paul; k=1942\n"));
     resultSet.close();
     // Only 2 rows came out of the table. If the value is 4, it means that the
@@ -126,28 +135,28 @@ public class ScannableTableTest {
     schema.add("beatles2", new BeatlesFilterableTable(buf, false));
     resultSet = statement.executeQuery(
         "select * from \"s\".\"beatles2\" where \"i\" = 4");
-    assertThat(OptiqAssert.toString(resultSet),
+    assertThat(CalciteAssert.toString(resultSet),
         equalTo("i=4; j=John; k=1940\ni=4; j=Paul; k=1942\n"));
     resultSet.close();
     assertThat(buf.toString(), equalTo("returnCount=4"));
     buf.setLength(0);
   }
 
-  /** A filter on a {@link net.hydromatic.optiq.ProjectableFilterableTable} with
-   * two columns. */
+  /** A filter on a {@link org.apache.calcite.schema.ProjectableFilterableTable}
+   * with two columns. */
   @Test public void testProjectableFilterable2() throws Exception {
     Connection connection =
         DriverManager.getConnection("jdbc:calcite:");
-    OptiqConnection optiqConnection =
-        connection.unwrap(OptiqConnection.class);
-    SchemaPlus rootSchema = optiqConnection.getRootSchema();
+    CalciteConnection calciteConnection =
+        connection.unwrap(CalciteConnection.class);
+    SchemaPlus rootSchema = calciteConnection.getRootSchema();
     SchemaPlus schema = rootSchema.add("s", new AbstractSchema());
     final StringBuilder buf = new StringBuilder();
     schema.add("beatles", new BeatlesProjectableFilterableTable(buf, true));
     final Statement statement = connection.createStatement();
     ResultSet resultSet = statement.executeQuery(
         "select * from \"s\".\"beatles\" where \"i\" = 4");
-    assertThat(OptiqAssert.toString(resultSet),
+    assertThat(CalciteAssert.toString(resultSet),
         equalTo("i=4; j=John; k=1940\ni=4; j=Paul; k=1942\n"));
     resultSet.close();
     // Only 2 rows came out of the table. If the value is 4, it means that the
@@ -160,21 +169,21 @@ public class ScannableTableTest {
     schema.add("beatles2", new BeatlesProjectableFilterableTable(buf, false));
     resultSet = statement.executeQuery(
         "select * from \"s\".\"beatles2\" where \"i\" = 4");
-    assertThat(OptiqAssert.toString(resultSet),
+    assertThat(CalciteAssert.toString(resultSet),
         equalTo("i=4; j=John; k=1940\ni=4; j=Paul; k=1942\n"));
     resultSet.close();
     assertThat(buf.toString(), equalTo("returnCount=4"));
     buf.setLength(0);
   }
 
-  /** A filter on a {@link net.hydromatic.optiq.ProjectableFilterableTable} with
-   * two columns, and a project in the query. */
+  /** A filter on a {@link org.apache.calcite.schema.ProjectableFilterableTable}
+   * with two columns, and a project in the query. */
   @Test public void testProjectableFilterable2WithProject() throws Exception {
     Connection connection =
         DriverManager.getConnection("jdbc:calcite:");
-    OptiqConnection optiqConnection =
-        connection.unwrap(OptiqConnection.class);
-    SchemaPlus rootSchema = optiqConnection.getRootSchema();
+    CalciteConnection calciteConnection =
+        connection.unwrap(CalciteConnection.class);
+    SchemaPlus rootSchema = calciteConnection.getRootSchema();
     SchemaPlus schema = rootSchema.add("s", new AbstractSchema());
     final StringBuilder buf = new StringBuilder();
     schema.add("beatles", new BeatlesProjectableFilterableTable(buf, true));
@@ -183,7 +192,7 @@ public class ScannableTableTest {
     final Statement statement = connection.createStatement();
     ResultSet resultSet = statement.executeQuery(
         "select \"k\",\"j\" from \"s\".\"beatles\" where \"i\" = 4");
-    assertThat(OptiqAssert.toString(resultSet),
+    assertThat(CalciteAssert.toString(resultSet),
         equalTo("k=1940; j=John\nk=1942; j=Paul\n"));
     resultSet.close();
     assertThat(buf.toString(), equalTo("returnCount=2, projects=[2, 1]"));
@@ -191,15 +200,15 @@ public class ScannableTableTest {
   }
 
   /** A filter and project on a
-   * {@link net.hydromatic.optiq.ProjectableFilterableTable}. The table
+   * {@link org.apache.calcite.schema.ProjectableFilterableTable}. The table
    * refuses to execute the filter, so Calcite should add a pull up and
    * transform the filter (projecting the column needed by the filter). */
   @Test public void testPFTableRefusesFilter() throws Exception {
     Connection connection =
         DriverManager.getConnection("jdbc:calcite:");
-    OptiqConnection optiqConnection =
-        connection.unwrap(OptiqConnection.class);
-    SchemaPlus rootSchema = optiqConnection.getRootSchema();
+    CalciteConnection calciteConnection =
+        connection.unwrap(CalciteConnection.class);
+    SchemaPlus rootSchema = calciteConnection.getRootSchema();
     SchemaPlus schema = rootSchema.add("s", new AbstractSchema());
     final StringBuilder buf = new StringBuilder();
     schema.add("beatles2", new BeatlesProjectableFilterableTable(buf, false));
@@ -209,7 +218,7 @@ public class ScannableTableTest {
     final Statement statement = connection.createStatement();
     ResultSet resultSet = statement.executeQuery(
         "select \"k\" from \"s\".\"beatles2\" where \"i\" = 4");
-    assertThat(OptiqAssert.toString(resultSet),
+    assertThat(CalciteAssert.toString(resultSet),
         equalTo(Bug.CALCITE_445_FIXED ? "k=1940\nk=1942\n" : ""));
     resultSet.close();
     assertThat(buf.toString(),

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/test/SqlFunctionsTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/SqlFunctionsTest.java b/core/src/test/java/org/apache/calcite/test/SqlFunctionsTest.java
index 9473cc0..293e886 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlFunctionsTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlFunctionsTest.java
@@ -14,20 +14,49 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.test;
+package org.apache.calcite.test;
 
-import net.hydromatic.avatica.ByteString;
-
-import net.hydromatic.optiq.runtime.*;
+import org.apache.calcite.avatica.ByteString;
+import org.apache.calcite.runtime.SqlFunctions;
+import org.apache.calcite.runtime.Utilities;
 
 import org.junit.Test;
 
-import java.util.*;
-
-import static net.hydromatic.optiq.runtime.SqlFunctions.*;
-
-import static org.hamcrest.CoreMatchers.*;
-import static org.junit.Assert.*;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import static org.apache.calcite.runtime.SqlFunctions.EPOCH_JULIAN;
+import static org.apache.calcite.runtime.SqlFunctions.TimeUnitRange;
+import static org.apache.calcite.runtime.SqlFunctions.charLength;
+import static org.apache.calcite.runtime.SqlFunctions.concat;
+import static org.apache.calcite.runtime.SqlFunctions.dateStringToUnixDate;
+import static org.apache.calcite.runtime.SqlFunctions.greater;
+import static org.apache.calcite.runtime.SqlFunctions.initcap;
+import static org.apache.calcite.runtime.SqlFunctions.intervalDayTimeToString;
+import static org.apache.calcite.runtime.SqlFunctions.intervalYearMonthToString;
+import static org.apache.calcite.runtime.SqlFunctions.lesser;
+import static org.apache.calcite.runtime.SqlFunctions.lower;
+import static org.apache.calcite.runtime.SqlFunctions.ltrim;
+import static org.apache.calcite.runtime.SqlFunctions.rtrim;
+import static org.apache.calcite.runtime.SqlFunctions.timeStringToUnixDate;
+import static org.apache.calcite.runtime.SqlFunctions.timestampStringToUnixDate;
+import static org.apache.calcite.runtime.SqlFunctions.trim;
+import static org.apache.calcite.runtime.SqlFunctions.unixDateExtract;
+import static org.apache.calcite.runtime.SqlFunctions.unixDateToString;
+import static org.apache.calcite.runtime.SqlFunctions.unixTimeToString;
+import static org.apache.calcite.runtime.SqlFunctions.unixTimestampToString;
+import static org.apache.calcite.runtime.SqlFunctions.upper;
+import static org.apache.calcite.runtime.SqlFunctions.ymdToJulian;
+import static org.apache.calcite.runtime.SqlFunctions.ymdToUnixDate;
+
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
 
 /**
  * Unit test for the methods in {@link SqlFunctions} that implement SQL

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/test/SqlLimitsTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/SqlLimitsTest.java b/core/src/test/java/org/apache/calcite/test/SqlLimitsTest.java
index cbc2ce2..8c58676 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlLimitsTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlLimitsTest.java
@@ -14,26 +14,31 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.test;
-
-import java.io.*;
-import java.text.*;
-import java.util.*;
-
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.reltype.RelDataTypeFactory;
-import org.eigenbase.reltype.RelDataTypeSystem;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.sql.type.*;
-
-import net.hydromatic.optiq.jdbc.JavaTypeFactoryImpl;
+package org.apache.calcite.test;
+
+import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.sql.SqlDialect;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.BasicSqlType;
+import org.apache.calcite.sql.type.SqlTypeName;
 
 import com.google.common.collect.ImmutableList;
 
 import org.junit.BeforeClass;
 import org.junit.Test;
 
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.text.DateFormat;
+import java.util.Calendar;
+import java.util.List;
+import java.util.Locale;
+import java.util.TimeZone;
+
 /**
  * Unit test for SQL limits.
  */

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/test/SqlTestGen.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/SqlTestGen.java b/core/src/test/java/org/apache/calcite/test/SqlTestGen.java
index 64e3dc7..c5ed8c0 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlTestGen.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlTestGen.java
@@ -14,17 +14,29 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.test;
-
-import java.io.*;
-import java.lang.reflect.*;
-import java.nio.charset.*;
-import java.util.*;
-
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.test.*;
-import org.eigenbase.sql.validate.*;
-import org.eigenbase.util.*;
+package org.apache.calcite.test;
+
+import org.apache.calcite.sql.SqlCollation;
+import org.apache.calcite.sql.test.DefaultSqlTestFactory;
+import org.apache.calcite.sql.test.DelegatingSqlTestFactory;
+import org.apache.calcite.sql.test.SqlTestFactory;
+import org.apache.calcite.sql.test.SqlTester;
+import org.apache.calcite.sql.test.SqlTesterImpl;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.util.BarfingInvocationHandler;
+import org.apache.calcite.util.Util;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.lang.reflect.Proxy;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.List;
 
 /**
  * Utility to generate a SQL script from validator test.
@@ -107,8 +119,7 @@ public class SqlTestGen {
     public SqlTester getTester() {
       final SqlTestFactory factory =
           new DelegatingSqlTestFactory(DefaultSqlTestFactory.INSTANCE) {
-            @Override
-            public SqlValidator getValidator(SqlTestFactory factory) {
+            @Override public SqlValidator getValidator(SqlTestFactory factory) {
               return (SqlValidator) Proxy.newProxyInstance(
                   SqlValidatorSpooler.class.getClassLoader(),
                   new Class[]{SqlValidator.class},
@@ -133,12 +144,10 @@ public class SqlTestGen {
           }
         }
 
-        @Override
-        public void checkColumnType(String sql, String expected) {
+        @Override public void checkColumnType(String sql, String expected) {
         }
 
-        @Override
-        public void checkResultType(String sql, String expected) {
+        @Override public void checkResultType(String sql, String expected) {
         }
 
         public void checkType(
@@ -163,19 +172,16 @@ public class SqlTestGen {
           // ignore it for now.
         }
 
-        @Override
-        public void checkIntervalConv(String sql, String expected) {
+        @Override public void checkIntervalConv(String sql, String expected) {
         }
 
-        @Override
-        public void checkRewrite(
+        @Override public void checkRewrite(
             SqlValidator validator,
             String query,
             String expectedRewrite) {
         }
 
-        @Override
-        public void checkFieldOrigin(
+        @Override public void checkFieldOrigin(
             String sql,
             String fieldOriginList) {
         }
@@ -184,7 +190,7 @@ public class SqlTestGen {
 
     /**
      * Handles the methods in
-     * {@link org.eigenbase.sql.validate.SqlValidator} that are called
+     * {@link org.apache.calcite.sql.validate.SqlValidator} that are called
      * from validator tests.
      */
     public static class MyInvocationHandler extends BarfingInvocationHandler {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/test/SqlToRelConverterExtendedTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/SqlToRelConverterExtendedTest.java b/core/src/test/java/org/apache/calcite/test/SqlToRelConverterExtendedTest.java
index e44277b..2deab9d 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlToRelConverterExtendedTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlToRelConverterExtendedTest.java
@@ -14,16 +14,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.test;
+package org.apache.calcite.test;
 
-import net.hydromatic.optiq.SchemaPlus;
-import net.hydromatic.optiq.runtime.Hook;
-import net.hydromatic.optiq.tools.Frameworks;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.RelOptCluster;
-import org.eigenbase.relopt.RelOptSchema;
-import org.eigenbase.test.SqlToRelConverterTest;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptSchema;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelShuttleImpl;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.rel.externalize.RelJsonReader;
+import org.apache.calcite.rel.externalize.RelJsonWriter;
+import org.apache.calcite.runtime.Hook;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.tools.Frameworks;
 
 import com.google.common.base.Function;
 
@@ -33,7 +35,7 @@ import org.junit.Before;
 import java.io.IOException;
 
 /**
- * Runs {@link org.eigenbase.test.SqlToRelConverterTest} with extensions.
+ * Runs {@link org.apache.calcite.test.SqlToRelConverterTest} with extensions.
  */
 public class SqlToRelConverterExtendedTest extends SqlToRelConverterTest {
   Hook.Closeable closeable;
@@ -64,8 +66,7 @@ public class SqlToRelConverterExtendedTest extends SqlToRelConverterTest {
     // Find the schema. If there are no tables in the plan, we won't need one.
     final RelOptSchema[] schemas = {null};
     rel.accept(new RelShuttleImpl() {
-      @Override
-      public RelNode visit(TableAccessRelBase scan) {
+      @Override public RelNode visit(TableScan scan) {
         schemas[0] = scan.getTable().getRelOptSchema();
         return super.visit(scan);
       }


[03/58] [abbrv] [partial] incubator-calcite git commit: [CALCITE-306] Standardize code style for "import package.*; "

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ExpressionVisitor.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ExpressionVisitor.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ExpressionVisitor.java
index 742785a..a30d608 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ExpressionVisitor.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ExpressionVisitor.java
@@ -14,9 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.expressions;
+package org.apache.calcite.linq4j.tree;
 
-import net.hydromatic.linq4j.function.Function;
+import org.apache.calcite.linq4j.function.Function;
 
 /**
  * Represents a visitor or rewriter for expression trees.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ExpressionWriter.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ExpressionWriter.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ExpressionWriter.java
index ae6b3bd..fd72330 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ExpressionWriter.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ExpressionWriter.java
@@ -14,10 +14,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.expressions;
+package org.apache.calcite.linq4j.tree;
 
 import java.lang.reflect.Type;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Iterator;
 
 /**
  * Converts an expression to Java code.
@@ -48,8 +50,7 @@ class ExpressionWriter {
     }
   }
 
-  @Override
-  public String toString() {
+  @Override public String toString() {
     return buf.toString();
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/tree/Expressions.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/Expressions.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/Expressions.java
index f37a549..68c963d 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/Expressions.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/Expressions.java
@@ -14,15 +14,29 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.expressions;
-
-import net.hydromatic.linq4j.Extensions;
-import net.hydromatic.linq4j.function.*;
-
-import java.lang.reflect.*;
+package org.apache.calcite.linq4j.tree;
+
+import org.apache.calcite.linq4j.Extensions;
+import org.apache.calcite.linq4j.function.Function;
+import org.apache.calcite.linq4j.function.Function0;
+import org.apache.calcite.linq4j.function.Function1;
+import org.apache.calcite.linq4j.function.Function2;
+import org.apache.calcite.linq4j.function.Predicate1;
+import org.apache.calcite.linq4j.function.Predicate2;
+
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Field;
+import java.lang.reflect.Member;
+import java.lang.reflect.Method;
+import java.lang.reflect.Type;
 import java.math.BigDecimal;
 import java.math.BigInteger;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
 
 /**
  * Utility methods for expressions, including a lot of factory methods.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/tree/FieldDeclaration.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/FieldDeclaration.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/FieldDeclaration.java
index d475d2b..c9774a6 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/FieldDeclaration.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/FieldDeclaration.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.expressions;
+package org.apache.calcite.linq4j.tree;
 
 import java.lang.reflect.Modifier;
 
@@ -34,8 +34,7 @@ public class FieldDeclaration extends MemberDeclaration {
     this.initializer = initializer;
   }
 
-  @Override
-  public MemberDeclaration accept(Visitor visitor) {
+  @Override public MemberDeclaration accept(Visitor visitor) {
     visitor = visitor.preVisit(this);
     // do not visit parameter - visit may not return a ParameterExpression
     final Expression initializer =
@@ -57,8 +56,7 @@ public class FieldDeclaration extends MemberDeclaration {
     writer.newlineAndIndent();
   }
 
-  @Override
-  public boolean equals(Object o) {
+  @Override public boolean equals(Object o) {
     if (this == o) {
       return true;
     }
@@ -82,8 +80,7 @@ public class FieldDeclaration extends MemberDeclaration {
     return true;
   }
 
-  @Override
-  public int hashCode() {
+  @Override public int hashCode() {
     int result = modifier;
     result = 31 * result + parameter.hashCode();
     result = 31 * result + (initializer != null ? initializer.hashCode() : 0);

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ForStatement.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ForStatement.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ForStatement.java
index cbf98c8..35ee1e1 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ForStatement.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ForStatement.java
@@ -14,9 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.expressions;
+package org.apache.calcite.linq4j.tree;
 
-import net.hydromatic.linq4j.Ord;
+import org.apache.calcite.linq4j.Ord;
 
 import java.util.List;
 
@@ -44,8 +44,7 @@ public class ForStatement extends Statement {
     this.body = body; // may be empty block, not null
   }
 
-  @Override
-  public ForStatement accept(Visitor visitor) {
+  @Override public ForStatement accept(Visitor visitor) {
     visitor = visitor.preVisit(this);
     List<DeclarationStatement> decls1 =
         Expressions.acceptDeclarations(declarations, visitor);
@@ -56,8 +55,7 @@ public class ForStatement extends Statement {
     return visitor.visit(this, decls1, condition1, post1, body1);
   }
 
-  @Override
-  void accept0(ExpressionWriter writer) {
+  @Override void accept0(ExpressionWriter writer) {
     writer.append("for (");
     for (Ord<DeclarationStatement> declaration : Ord.zip(declarations)) {
       declaration.e.accept2(writer, declaration.i == 0);
@@ -73,8 +71,7 @@ public class ForStatement extends Statement {
     writer.append(") ").append(Blocks.toBlock(body));
   }
 
-  @Override
-  public boolean equals(Object o) {
+  @Override public boolean equals(Object o) {
     if (this == o) {
       return true;
     }
@@ -104,8 +101,7 @@ public class ForStatement extends Statement {
     return true;
   }
 
-  @Override
-  public int hashCode() {
+  @Override public int hashCode() {
     int result = hash;
     if (result == 0) {
       result = super.hashCode();

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/tree/FunctionExpression.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/FunctionExpression.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/FunctionExpression.java
index ff12b5b..705693c 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/FunctionExpression.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/FunctionExpression.java
@@ -14,12 +14,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.expressions;
+package org.apache.calcite.linq4j.tree;
 
-import net.hydromatic.linq4j.function.*;
+import org.apache.calcite.linq4j.function.Function;
+import org.apache.calcite.linq4j.function.Functions;
 
-import java.lang.reflect.*;
-import java.util.*;
+import java.lang.reflect.InvocationHandler;
+import java.lang.reflect.Method;
+import java.lang.reflect.Proxy;
+import java.lang.reflect.Type;
+import java.lang.reflect.TypeVariable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
 
 /**
  * Represents a strongly typed lambda expression as a data structure in the form
@@ -42,8 +49,8 @@ public final class FunctionExpression<F extends Function<?>>
       List<ParameterExpression> parameterList) {
     super(ExpressionType.Lambda, type);
     assert type != null : "type should not be null";
-    assert function != null || body != null : "both function and body should "
-        + "not be null";
+    assert function != null || body != null
+        : "both function and body should not be null";
     assert parameterList != null : "parameterList should not be null";
     this.function = function;
     this.body = body;
@@ -60,8 +67,7 @@ public final class FunctionExpression<F extends Function<?>>
     this(type, null, body, parameters);
   }
 
-  @Override
-  public Expression accept(Visitor visitor) {
+  @Override public Expression accept(Visitor visitor) {
     visitor = visitor.preVisit(this);
     BlockStatement body = this.body.accept(visitor);
     return visitor.visit(this, body);
@@ -99,8 +105,7 @@ public final class FunctionExpression<F extends Function<?>>
     return dynamicFunction;
   }
 
-  @Override
-  void accept(ExpressionWriter writer, int lprec, int rprec) {
+  @Override void accept(ExpressionWriter writer, int lprec, int rprec) {
     // "new Function1() {
     //    public Result apply(T1 p1, ...) {
     //        <body>
@@ -196,14 +201,13 @@ public final class FunctionExpression<F extends Function<?>>
   }
 
   private String getAbstractMethodName() {
-    if (type.toString().contains("OptiqFlatMapFunction")) {
+    if (type.toString().contains("CalciteFlatMapFunction")) {
       return "call"; // FIXME
     }
     return "apply";
   }
 
-  @Override
-  public boolean equals(Object o) {
+  @Override public boolean equals(Object o) {
     if (this == o) {
       return true;
     }
@@ -230,8 +234,7 @@ public final class FunctionExpression<F extends Function<?>>
     return true;
   }
 
-  @Override
-  public int hashCode() {
+  @Override public int hashCode() {
     int result = hash;
     if (result == 0) {
       result = super.hashCode();

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/tree/GotoExpressionKind.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/GotoExpressionKind.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/GotoExpressionKind.java
index 853ebfc..04ce0ad 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/GotoExpressionKind.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/GotoExpressionKind.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.expressions;
+package org.apache.calcite.linq4j.tree;
 
 /**
  * Specifies what kind of jump a {@link GotoStatement} represents.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/tree/GotoStatement.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/GotoStatement.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/GotoStatement.java
index 47897a2..18549d8 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/GotoStatement.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/GotoStatement.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.expressions;
+package org.apache.calcite.linq4j.tree;
 
 /**
  * Represents an unconditional jump. This includes return statements, break and
@@ -52,16 +52,14 @@ public class GotoStatement extends Statement {
     }
   }
 
-  @Override
-  public Statement accept(Visitor visitor) {
+  @Override public Statement accept(Visitor visitor) {
     visitor = visitor.preVisit(this);
     Expression expression1 =
         expression == null ? null : expression.accept(visitor);
     return visitor.visit(this, expression1);
   }
 
-  @Override
-  void accept0(ExpressionWriter writer) {
+  @Override void accept0(ExpressionWriter writer) {
     writer.append(kind.prefix);
     if (labelTarget != null) {
       writer.append(' ').append(labelTarget.name);
@@ -84,8 +82,7 @@ public class GotoStatement extends Statement {
     writer.append(';').newlineAndIndent();
   }
 
-  @Override
-  public Object evaluate(Evaluator evaluator) {
+  @Override public Object evaluate(Evaluator evaluator) {
     switch (kind) {
     case Return:
     case Sequence:
@@ -97,8 +94,7 @@ public class GotoStatement extends Statement {
     }
   }
 
-  @Override
-  public boolean equals(Object o) {
+  @Override public boolean equals(Object o) {
     if (this == o) {
       return true;
     }
@@ -126,8 +122,7 @@ public class GotoStatement extends Statement {
     return true;
   }
 
-  @Override
-  public int hashCode() {
+  @Override public int hashCode() {
     int result = super.hashCode();
     result = 31 * result + kind.hashCode();
     result = 31 * result + (labelTarget != null ? labelTarget.hashCode() : 0);

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/tree/IndexExpression.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/IndexExpression.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/IndexExpression.java
index ec03dee..c61b120 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/IndexExpression.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/IndexExpression.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.expressions;
+package org.apache.calcite.linq4j.tree;
 
 import java.util.List;
 
@@ -34,8 +34,7 @@ public class IndexExpression extends Expression {
     this.indexExpressions = indexExpressions;
   }
 
-  @Override
-  public Expression accept(Visitor visitor) {
+  @Override public Expression accept(Visitor visitor) {
     visitor = visitor.preVisit(this);
     Expression array = this.array.accept(visitor);
     List<Expression> indexExpressions = Expressions.acceptExpressions(
@@ -44,14 +43,12 @@ public class IndexExpression extends Expression {
   }
 
 
-  @Override
-  void accept(ExpressionWriter writer, int lprec, int rprec) {
+  @Override void accept(ExpressionWriter writer, int lprec, int rprec) {
     array.accept(writer, lprec, nodeType.lprec);
     writer.list("[", ", ", "]", indexExpressions);
   }
 
-  @Override
-  public boolean equals(Object o) {
+  @Override public boolean equals(Object o) {
     if (this == o) {
       return true;
     }
@@ -74,8 +71,7 @@ public class IndexExpression extends Expression {
     return true;
   }
 
-  @Override
-  public int hashCode() {
+  @Override public int hashCode() {
     int result = super.hashCode();
     result = 31 * result + array.hashCode();
     result = 31 * result + indexExpressions.hashCode();

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/tree/InvocationExpression.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/InvocationExpression.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/InvocationExpression.java
index d3fce11..666ab07 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/InvocationExpression.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/InvocationExpression.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.expressions;
+package org.apache.calcite.linq4j.tree;
 
 /**
  * Represents an expression that applies a delegate or lambda expression to a
@@ -25,8 +25,7 @@ public class InvocationExpression extends Expression {
     super(nodeType, type);
   }
 
-  @Override
-  public Expression accept(Visitor visitor) {
+  @Override public Expression accept(Visitor visitor) {
     return visitor.visit(this);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/tree/LabelStatement.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/LabelStatement.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/LabelStatement.java
index c8d7d4b..3b00b1e 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/LabelStatement.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/LabelStatement.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.expressions;
+package org.apache.calcite.linq4j.tree;
 
 /**
  * Represents a label, which can be put in any {@link Expression} context. If it
@@ -31,13 +31,11 @@ public class LabelStatement extends Statement {
     this.defaultValue = defaultValue;
   }
 
-  @Override
-  public LabelStatement accept(Visitor visitor) {
+  @Override public LabelStatement accept(Visitor visitor) {
     return visitor.visit(this);
   }
 
-  @Override
-  public boolean equals(Object o) {
+  @Override public boolean equals(Object o) {
     if (this == o) {
       return true;
     }
@@ -58,8 +56,7 @@ public class LabelStatement extends Statement {
     return true;
   }
 
-  @Override
-  public int hashCode() {
+  @Override public int hashCode() {
     int result = super.hashCode();
     result = 31 * result + (defaultValue != null ? defaultValue.hashCode() : 0);
     return result;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/tree/LabelTarget.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/LabelTarget.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/LabelTarget.java
index 0b38e11..1c62748 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/LabelTarget.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/LabelTarget.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.expressions;
+package org.apache.calcite.linq4j.tree;
 
 /**
  * Used to represent the target of a {@link GotoStatement}.
@@ -26,8 +26,7 @@ public class LabelTarget {
     this.name = name;
   }
 
-  @Override
-  public boolean equals(Object o) {
+  @Override public boolean equals(Object o) {
     if (this == o) {
       return true;
     }
@@ -44,8 +43,7 @@ public class LabelTarget {
     return true;
   }
 
-  @Override
-  public int hashCode() {
+  @Override public int hashCode() {
     return name != null ? name.hashCode() : 0;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/tree/LambdaExpression.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/LambdaExpression.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/LambdaExpression.java
index b10ad9f..d406e8f 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/LambdaExpression.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/LambdaExpression.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.expressions;
+package org.apache.calcite.linq4j.tree;
 
 /**
  * Describes a lambda expression. This captures a block of code that is similar
@@ -25,8 +25,7 @@ public class LambdaExpression extends Expression {
     super(nodeType, type);
   }
 
-  @Override
-  public Expression accept(Visitor visitor) {
+  @Override public Expression accept(Visitor visitor) {
     return visitor.visit(this);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ListInitExpression.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ListInitExpression.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ListInitExpression.java
index d42f7e7..bf22c11 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ListInitExpression.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ListInitExpression.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.expressions;
+package org.apache.calcite.linq4j.tree;
 
 /**
  * Represents a constructor call that has a collection initializer.
@@ -24,8 +24,7 @@ public class ListInitExpression extends Expression {
     super(nodeType, type);
   }
 
-  @Override
-  public Expression accept(Visitor visitor) {
+  @Override public Expression accept(Visitor visitor) {
     return visitor.visit(this);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/tree/MemberAssignment.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/MemberAssignment.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/MemberAssignment.java
index f6e92de..c3d9980 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/MemberAssignment.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/MemberAssignment.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.expressions;
+package org.apache.calcite.linq4j.tree;
 
 /**
  * Represents assignment operation for a field or property of an object.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/tree/MemberBinding.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/MemberBinding.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/MemberBinding.java
index d5790ce..2b7a691 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/MemberBinding.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/MemberBinding.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.expressions;
+package org.apache.calcite.linq4j.tree;
 
 /**
  * Provides the base class from which the classes that represent bindings that

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/tree/MemberDeclaration.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/MemberDeclaration.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/MemberDeclaration.java
index 1a076a9..c3c10b4 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/MemberDeclaration.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/MemberDeclaration.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.expressions;
+package org.apache.calcite.linq4j.tree;
 
 /**
  * Declaration of a member of a class.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/tree/MemberExpression.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/MemberExpression.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/MemberExpression.java
index 7027c18..e8d86d8 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/MemberExpression.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/MemberExpression.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.expressions;
+package org.apache.calcite.linq4j.tree;
 
 import java.lang.reflect.Field;
 import java.lang.reflect.Modifier;
@@ -39,8 +39,7 @@ public class MemberExpression extends Expression {
     this.field = field;
   }
 
-  @Override
-  public Expression accept(Visitor visitor) {
+  @Override public Expression accept(Visitor visitor) {
     visitor = visitor.preVisit(this);
     Expression expression1 = expression == null
         ? null
@@ -59,8 +58,7 @@ public class MemberExpression extends Expression {
     }
   }
 
-  @Override
-  void accept(ExpressionWriter writer, int lprec, int rprec) {
+  @Override void accept(ExpressionWriter writer, int lprec, int rprec) {
     if (writer.requireParentheses(this, lprec, rprec)) {
       return;
     }
@@ -73,8 +71,7 @@ public class MemberExpression extends Expression {
     writer.append('.').append(field.getName());
   }
 
-  @Override
-  public boolean equals(Object o) {
+  @Override public boolean equals(Object o) {
     if (this == o) {
       return true;
     }
@@ -98,8 +95,7 @@ public class MemberExpression extends Expression {
     return true;
   }
 
-  @Override
-  public int hashCode() {
+  @Override public int hashCode() {
     int result = super.hashCode();
     result = 31 * result + (expression != null ? expression.hashCode() : 0);
     result = 31 * result + field.hashCode();

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/tree/MemberInitExpression.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/MemberInitExpression.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/MemberInitExpression.java
index 461edf3..79b4c46 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/MemberInitExpression.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/MemberInitExpression.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.expressions;
+package org.apache.calcite.linq4j.tree;
 
 /**
  * Represents calling a constructor and initializing one or more members of the
@@ -25,8 +25,7 @@ public class MemberInitExpression extends Expression {
     super(ExpressionType.MemberInit, Void.TYPE);
   }
 
-  @Override
-  public Expression accept(Visitor visitor) {
+  @Override public Expression accept(Visitor visitor) {
     return visitor.visit(this);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/tree/MemberListBinding.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/MemberListBinding.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/MemberListBinding.java
index 7f86361..84173c9 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/MemberListBinding.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/MemberListBinding.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.expressions;
+package org.apache.calcite.linq4j.tree;
 
 /**
  * Represents initializing the elements of a collection member of a newly

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/tree/MemberMemberBinding.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/MemberMemberBinding.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/MemberMemberBinding.java
index b05506a..4ca828e 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/MemberMemberBinding.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/MemberMemberBinding.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.expressions;
+package org.apache.calcite.linq4j.tree;
 
 /**
  * Represents initializing members of a member of a newly created object.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/tree/MethodCallExpression.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/MethodCallExpression.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/MethodCallExpression.java
index fa045ff..2dc2ee8 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/MethodCallExpression.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/MethodCallExpression.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.expressions;
+package org.apache.calcite.linq4j.tree;
 
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
@@ -52,8 +52,7 @@ public class MethodCallExpression extends Expression {
     this(method.getGenericReturnType(), method, targetExpression, expressions);
   }
 
-  @Override
-  public Expression accept(Visitor visitor) {
+  @Override public Expression accept(Visitor visitor) {
     visitor = visitor.preVisit(this);
     Expression targetExpression = Expressions.accept(this.targetExpression,
         visitor);
@@ -62,8 +61,7 @@ public class MethodCallExpression extends Expression {
     return visitor.visit(this, targetExpression, expressions);
   }
 
-  @Override
-  public Object evaluate(Evaluator evaluator) {
+  @Override public Object evaluate(Evaluator evaluator) {
     final Object target;
     if (targetExpression == null) {
       target = null;
@@ -84,8 +82,7 @@ public class MethodCallExpression extends Expression {
     }
   }
 
-  @Override
-  void accept(ExpressionWriter writer, int lprec, int rprec) {
+  @Override void accept(ExpressionWriter writer, int lprec, int rprec) {
     if (writer.requireParentheses(this, lprec, rprec)) {
       return;
     }
@@ -107,8 +104,7 @@ public class MethodCallExpression extends Expression {
     writer.append(')');
   }
 
-  @Override
-  public boolean equals(Object o) {
+  @Override public boolean equals(Object o) {
     if (this == o) {
       return true;
     }
@@ -135,8 +131,7 @@ public class MethodCallExpression extends Expression {
     return true;
   }
 
-  @Override
-  public int hashCode() {
+  @Override public int hashCode() {
     int result = hash;
     if (result == 0) {
       result = super.hashCode();

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/tree/MethodDeclaration.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/MethodDeclaration.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/MethodDeclaration.java
index 9ab611b..0e4c99a 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/MethodDeclaration.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/MethodDeclaration.java
@@ -14,10 +14,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.expressions;
+package org.apache.calcite.linq4j.tree;
 
-import net.hydromatic.linq4j.function.Function1;
-import net.hydromatic.linq4j.function.Functions;
+import org.apache.calcite.linq4j.function.Function1;
+import org.apache.calcite.linq4j.function.Functions;
 
 import java.lang.reflect.Modifier;
 import java.lang.reflect.Type;
@@ -46,8 +46,7 @@ public class MethodDeclaration extends MemberDeclaration {
     this.body = body;
   }
 
-  @Override
-  public MemberDeclaration accept(Visitor visitor) {
+  @Override public MemberDeclaration accept(Visitor visitor) {
     visitor = visitor.preVisit(this);
     // do not visit parameters
     final BlockStatement body = this.body.accept(visitor);
@@ -76,8 +75,7 @@ public class MethodDeclaration extends MemberDeclaration {
     writer.newlineAndIndent();
   }
 
-  @Override
-  public boolean equals(Object o) {
+  @Override public boolean equals(Object o) {
     if (this == o) {
       return true;
     }
@@ -106,8 +104,7 @@ public class MethodDeclaration extends MemberDeclaration {
     return true;
   }
 
-  @Override
-  public int hashCode() {
+  @Override public int hashCode() {
     int result = modifier;
     result = 31 * result + name.hashCode();
     result = 31 * result + resultType.hashCode();

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/tree/NewArrayExpression.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/NewArrayExpression.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/NewArrayExpression.java
index dced9d9..de45a5e 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/NewArrayExpression.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/NewArrayExpression.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.expressions;
+package org.apache.calcite.linq4j.tree;
 
 import java.lang.reflect.Type;
 import java.util.List;
@@ -40,8 +40,7 @@ public class NewArrayExpression extends Expression {
     this.expressions = expressions;
   }
 
-  @Override
-  public Expression accept(Visitor visitor) {
+  @Override public Expression accept(Visitor visitor) {
     visitor = visitor.preVisit(this);
     List<Expression> expressions =
         this.expressions == null
@@ -51,8 +50,7 @@ public class NewArrayExpression extends Expression {
     return visitor.visit(this, dimension, bound, expressions);
   }
 
-  @Override
-  void accept(ExpressionWriter writer, int lprec, int rprec) {
+  @Override void accept(ExpressionWriter writer, int lprec, int rprec) {
     writer.append("new ").append(Types.getComponentTypeN(type));
     for (int i = 0; i < dimension; i++) {
       if (i == 0 && bound != null) {
@@ -66,8 +64,7 @@ public class NewArrayExpression extends Expression {
     }
   }
 
-  @Override
-  public boolean equals(Object o) {
+  @Override public boolean equals(Object o) {
     if (this == o) {
       return true;
     }
@@ -94,8 +91,7 @@ public class NewArrayExpression extends Expression {
     return true;
   }
 
-  @Override
-  public int hashCode() {
+  @Override public int hashCode() {
     int result = hash;
     if (result == 0) {
       result = super.hashCode();

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/tree/NewExpression.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/NewExpression.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/NewExpression.java
index 8620983..70ca418 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/NewExpression.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/NewExpression.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.expressions;
+package org.apache.calcite.linq4j.tree;
 
 import java.lang.reflect.Type;
 import java.util.List;
@@ -42,8 +42,7 @@ public class NewExpression extends Expression {
     this.memberDeclarations = memberDeclarations;
   }
 
-  @Override
-  public Expression accept(Visitor visitor) {
+  @Override public Expression accept(Visitor visitor) {
     visitor = visitor.preVisit(this);
     final List<Expression> arguments = Expressions.acceptExpressions(
         this.arguments, visitor);
@@ -52,16 +51,14 @@ public class NewExpression extends Expression {
     return visitor.visit(this, arguments, memberDeclarations);
   }
 
-  @Override
-  void accept(ExpressionWriter writer, int lprec, int rprec) {
+  @Override void accept(ExpressionWriter writer, int lprec, int rprec) {
     writer.append("new ").append(type).list("(\n", ",\n", ")", arguments);
     if (memberDeclarations != null) {
       writer.list("{\n", "", "}", memberDeclarations);
     }
   }
 
-  @Override
-  public boolean equals(Object o) {
+  @Override public boolean equals(Object o) {
     if (this == o) {
       return true;
     }
@@ -90,8 +87,7 @@ public class NewExpression extends Expression {
     return true;
   }
 
-  @Override
-  public int hashCode() {
+  @Override public int hashCode() {
     int result = hash;
     if (result == 0) {
       result = super.hashCode();

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/tree/Node.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/Node.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/Node.java
index 648215c..7c04f08 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/Node.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/Node.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.expressions;
+package org.apache.calcite.linq4j.tree;
 
 /**
  * <p>Parse tree node.</p>

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/tree/OptimizeVisitor.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/OptimizeVisitor.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/OptimizeVisitor.java
index c3bc31d..f0bed0f 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/OptimizeVisitor.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/OptimizeVisitor.java
@@ -14,14 +14,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.expressions;
+package org.apache.calcite.linq4j.tree;
 
 import java.lang.reflect.Method;
 import java.lang.reflect.Modifier;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.EnumMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
 
-import static net.hydromatic.linq4j.expressions.ExpressionType.Equal;
-import static net.hydromatic.linq4j.expressions.ExpressionType.NotEqual;
+import static org.apache.calcite.linq4j.tree.ExpressionType.Equal;
+import static org.apache.calcite.linq4j.tree.ExpressionType.NotEqual;
 
 /**
  * Visitor that optimizes expressions.
@@ -75,8 +80,7 @@ public class OptimizeVisitor extends Visitor {
   private static final Method BOOLEAN_VALUEOF_BOOL =
       Types.lookupMethod(Boolean.class, "valueOf", boolean.class);
 
-  @Override
-  public Expression visit(
+  @Override public Expression visit(
       TernaryExpression ternary,
       Expression expression0,
       Expression expression1,
@@ -151,8 +155,7 @@ public class OptimizeVisitor extends Visitor {
     return super.visit(ternary, expression0, expression1, expression2);
   }
 
-  @Override
-  public Expression visit(
+  @Override public Expression visit(
       BinaryExpression binary,
       Expression expression0,
       Expression expression1) {
@@ -266,8 +269,7 @@ public class OptimizeVisitor extends Visitor {
     return null;
   }
 
-  @Override
-  public Expression visit(UnaryExpression unaryExpression,
+  @Override public Expression visit(UnaryExpression unaryExpression,
       Expression expression) {
     switch (unaryExpression.getNodeType()) {
     case Convert:
@@ -301,8 +303,7 @@ public class OptimizeVisitor extends Visitor {
     return super.visit(unaryExpression, expression);
   }
 
-  @Override
-  public Statement visit(ConditionalStatement conditionalStatement,
+  @Override public Statement visit(ConditionalStatement conditionalStatement,
       List<Node> list) {
     // if (false) { <-- remove branch
     // } if (true) { <-- stop here
@@ -361,8 +362,7 @@ public class OptimizeVisitor extends Visitor {
     return super.visit(conditionalStatement, newList);
   }
 
-  @Override
-  public Expression visit(MethodCallExpression methodCallExpression,
+  @Override public Expression visit(MethodCallExpression methodCallExpression,
       Expression targetExpression,
       List<Expression> expressions) {
     if (BOOLEAN_VALUEOF_BOOL.equals(methodCallExpression.method)) {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ParameterExpression.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ParameterExpression.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ParameterExpression.java
index 9eaafe7..a437a46 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ParameterExpression.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ParameterExpression.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.expressions;
+package org.apache.calcite.linq4j.tree;
 
 import java.lang.reflect.Modifier;
 import java.lang.reflect.Type;
@@ -43,8 +43,7 @@ public class ParameterExpression extends Expression {
     this.name = name;
   }
 
-  @Override
-  public Expression accept(Visitor visitor) {
+  @Override public Expression accept(Visitor visitor) {
     return visitor.visit(this);
   }
 
@@ -52,8 +51,7 @@ public class ParameterExpression extends Expression {
     return evaluator.peek(this);
   }
 
-  @Override
-  void accept(ExpressionWriter writer, int lprec, int rprec) {
+  @Override void accept(ExpressionWriter writer, int lprec, int rprec) {
     writer.append(name);
   }
 
@@ -63,20 +61,15 @@ public class ParameterExpression extends Expression {
 
   String declString(Type type) {
     final String modifiers = Modifier.toString(modifier);
-    return modifiers
-        + (modifiers.isEmpty() ? "" : " ")
-        + Types.className(type)
-        + " "
-        + name;
+    return modifiers + (modifiers.isEmpty() ? "" : " ") + Types.className(type)
+        + " " + name;
   }
 
-  @Override
-  public boolean equals(Object o) {
+  @Override public boolean equals(Object o) {
     return this == o;
   }
 
-  @Override
-  public int hashCode() {
+  @Override public int hashCode() {
     return System.identityHashCode(this);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/tree/Primitive.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/Primitive.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/Primitive.java
index 9663dda..67648dd 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/Primitive.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/Primitive.java
@@ -14,14 +14,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.expressions;
+package org.apache.calcite.linq4j.tree;
 
 import java.lang.reflect.Array;
 import java.lang.reflect.Field;
 import java.lang.reflect.Type;
 import java.sql.ResultSet;
 import java.sql.SQLException;
-import java.util.*;
+import java.util.AbstractList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
 
 /**
  * Enumeration of Java's primitive types.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/tree/PseudoField.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/PseudoField.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/PseudoField.java
index 2f632c3..1cb0708 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/PseudoField.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/PseudoField.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.expressions;
+package org.apache.calcite.linq4j.tree;
 
 import java.lang.reflect.Type;
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ReflectedPseudoField.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ReflectedPseudoField.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ReflectedPseudoField.java
index 5f0052a..d08b06c 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ReflectedPseudoField.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ReflectedPseudoField.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.expressions;
+package org.apache.calcite.linq4j.tree;
 
 import java.lang.reflect.Field;
 import java.lang.reflect.Type;
@@ -50,8 +50,7 @@ public class ReflectedPseudoField implements PseudoField {
     return field.getDeclaringClass();
   }
 
-  @Override
-  public boolean equals(Object o) {
+  @Override public boolean equals(Object o) {
     if (this == o) {
       return true;
     }
@@ -68,8 +67,9 @@ public class ReflectedPseudoField implements PseudoField {
     return true;
   }
 
-  @Override
-  public int hashCode() {
+  @Override public int hashCode() {
     return field.hashCode();
   }
 }
+
+// End ReflectedPseudoField.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/tree/Statement.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/Statement.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/Statement.java
index 9f5163e..99e17f9 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/Statement.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/Statement.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.expressions;
+package org.apache.calcite.linq4j.tree;
 
 import java.lang.reflect.Type;
 
@@ -26,17 +26,15 @@ public abstract class Statement extends AbstractNode {
     super(nodeType, type);
   }
 
-  @Override
-  final void accept(ExpressionWriter writer, int lprec, int rprec) {
+  @Override final void accept(ExpressionWriter writer, int lprec, int rprec) {
     assert lprec == 0;
     assert rprec == 0;
     accept0(writer);
   }
 
-  @Override
   // Make return type more specific. A statement can only become a different
   // kind of statement; it can't become an expression.
-  public abstract Statement accept(Visitor visitor);
+  @Override public abstract Statement accept(Visitor visitor);
 }
 
 // End Statement.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/tree/SwitchCase.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/SwitchCase.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/SwitchCase.java
index b2e81bb..863fa8b 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/SwitchCase.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/SwitchCase.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.expressions;
+package org.apache.calcite.linq4j.tree;
 
 /**
  * Represents one case of a {@link SwitchStatement}.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/tree/SwitchStatement.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/SwitchStatement.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/SwitchStatement.java
index f8a7c14..b0fa124 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/SwitchStatement.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/SwitchStatement.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.expressions;
+package org.apache.calcite.linq4j.tree;
 
 /**
  * Represents a control expression that handles multiple selections by passing
@@ -25,8 +25,7 @@ public class SwitchStatement extends Statement {
     super(nodeType, Void.TYPE);
   }
 
-  @Override
-  public Statement accept(Visitor visitor) {
+  @Override public Statement accept(Visitor visitor) {
     return visitor.visit(this);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/tree/TernaryExpression.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/TernaryExpression.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/TernaryExpression.java
index 928bb40..26cf857 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/TernaryExpression.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/TernaryExpression.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.expressions;
+package org.apache.calcite.linq4j.tree;
 
 import java.lang.reflect.Type;
 
@@ -37,8 +37,7 @@ public class TernaryExpression extends Expression {
     this.expression2 = expression2;
   }
 
-  @Override
-  public Expression accept(Visitor visitor) {
+  @Override public Expression accept(Visitor visitor) {
     visitor = visitor.preVisit(this);
     Expression expression0 = this.expression0.accept(visitor);
     Expression expression1 = this.expression1.accept(visitor);
@@ -57,8 +56,7 @@ public class TernaryExpression extends Expression {
     expression2.accept(writer, nodeType.rprec, rprec);
   }
 
-  @Override
-  public boolean equals(Object o) {
+  @Override public boolean equals(Object o) {
     if (this == o) {
       return true;
     }
@@ -84,8 +82,7 @@ public class TernaryExpression extends Expression {
     return true;
   }
 
-  @Override
-  public int hashCode() {
+  @Override public int hashCode() {
     int result = super.hashCode();
     result = 31 * result + expression0.hashCode();
     result = 31 * result + expression1.hashCode();

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ThrowStatement.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ThrowStatement.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ThrowStatement.java
index 6e2ec84..037342c 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ThrowStatement.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ThrowStatement.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.expressions;
+package org.apache.calcite.linq4j.tree;
 
 /**
  * Represents a {@code throw} statement.
@@ -27,20 +27,17 @@ public class ThrowStatement extends Statement {
     this.expression = expression;
   }
 
-  @Override
-  public Statement accept(Visitor visitor) {
+  @Override public Statement accept(Visitor visitor) {
     visitor = visitor.preVisit(this);
     Expression expression = this.expression.accept(visitor);
     return visitor.visit(this, expression);
   }
 
-  @Override
-  void accept0(ExpressionWriter writer) {
+  @Override void accept0(ExpressionWriter writer) {
     writer.append("throw ").append(expression).append(';').newlineAndIndent();
   }
 
-  @Override
-  public boolean equals(Object o) {
+  @Override public boolean equals(Object o) {
     if (this == o) {
       return true;
     }
@@ -61,8 +58,7 @@ public class ThrowStatement extends Statement {
     return true;
   }
 
-  @Override
-  public int hashCode() {
+  @Override public int hashCode() {
     int result = super.hashCode();
     result = 31 * result + (expression != null ? expression.hashCode() : 0);
     return result;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/tree/TryStatement.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/TryStatement.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/TryStatement.java
index 0ea7afa..518ac34 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/TryStatement.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/TryStatement.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.expressions;
+package org.apache.calcite.linq4j.tree;
 
 import java.util.List;
 
@@ -36,13 +36,11 @@ public class TryStatement extends Statement {
     this.fynally = fynally;
   }
 
-  @Override
-  public Statement accept(Visitor visitor) {
+  @Override public Statement accept(Visitor visitor) {
     return visitor.visit(this);
   }
 
-  @Override
-  void accept0(ExpressionWriter writer) {
+  @Override void accept0(ExpressionWriter writer) {
     writer.append("try ").append(Blocks.toBlock(body));
     for (CatchBlock catchBlock : catchBlocks) {
       writer.backUp();
@@ -55,8 +53,7 @@ public class TryStatement extends Statement {
     }
   }
 
-  @Override
-  public boolean equals(Object o) {
+  @Override public boolean equals(Object o) {
     if (this == o) {
       return true;
     }
@@ -83,8 +80,7 @@ public class TryStatement extends Statement {
     return true;
   }
 
-  @Override
-  public int hashCode() {
+  @Override public int hashCode() {
     int result = super.hashCode();
     result = 31 * result + body.hashCode();
     result = 31 * result + catchBlocks.hashCode();

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/tree/TypeBinaryExpression.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/TypeBinaryExpression.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/TypeBinaryExpression.java
index eac2ddf..526e1f3 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/TypeBinaryExpression.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/TypeBinaryExpression.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.expressions;
+package org.apache.calcite.linq4j.tree;
 
 import java.lang.reflect.Type;
 
@@ -33,8 +33,7 @@ public class TypeBinaryExpression extends Expression {
     this.type = type;
   }
 
-  @Override
-  public Expression accept(Visitor visitor) {
+  @Override public Expression accept(Visitor visitor) {
     visitor = visitor.preVisit(this);
     Expression expression = this.expression.accept(visitor);
     return visitor.visit(this, expression);
@@ -49,8 +48,7 @@ public class TypeBinaryExpression extends Expression {
     writer.append(type);
   }
 
-  @Override
-  public boolean equals(Object o) {
+  @Override public boolean equals(Object o) {
     if (this == o) {
       return true;
     }
@@ -73,8 +71,7 @@ public class TypeBinaryExpression extends Expression {
     return true;
   }
 
-  @Override
-  public int hashCode() {
+  @Override public int hashCode() {
     int result = super.hashCode();
     result = 31 * result + expression.hashCode();
     result = 31 * result + (type != null ? type.hashCode() : 0);

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/tree/Types.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/Types.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/Types.java
index 8b67ec6..01bc947 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/Types.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/Types.java
@@ -14,12 +14,24 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.expressions;
-
-import net.hydromatic.linq4j.Enumerator;
-
-import java.lang.reflect.*;
-import java.util.*;
+package org.apache.calcite.linq4j.tree;
+
+import org.apache.calcite.linq4j.Enumerator;
+
+import java.lang.reflect.Array;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Field;
+import java.lang.reflect.GenericArrayType;
+import java.lang.reflect.Method;
+import java.lang.reflect.ParameterizedType;
+import java.lang.reflect.Type;
+import java.lang.reflect.TypeVariable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
 
 /**
  * Utilities for converting between {@link Expression}, {@link Type} and
@@ -42,8 +54,8 @@ public abstract class Types {
 
   /**
    * Returns the element type of a {@link Collection}, {@link Iterable}
-   * (including {@link net.hydromatic.linq4j.Queryable Queryable} and
-   * {@link net.hydromatic.linq4j.Enumerable Enumerable}), {@link Iterator},
+   * (including {@link org.apache.calcite.linq4j.Queryable Queryable} and
+   * {@link org.apache.calcite.linq4j.Enumerable Enumerable}), {@link Iterator},
    * {@link Enumerator}, or an array.
    *
    * <p>Returns null if the type is not one of these.</p>
@@ -330,10 +342,8 @@ public abstract class Types {
           return method;
         }
       }
-      throw new RuntimeException(
-          "while resolving method '" + methodName
-          + Arrays.toString(argumentTypes)
-          + "' in class " + clazz, e);
+      throw new RuntimeException("while resolving method '" + methodName
+          + Arrays.toString(argumentTypes) + "' in class " + clazz, e);
     }
   }
 
@@ -522,8 +532,7 @@ public abstract class Types {
       }
     }
 
-    @Override
-    public String toString() {
+    @Override public String toString() {
       final StringBuilder buf = new StringBuilder();
       buf.append(className(rawType));
       buf.append("<");

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/tree/UnaryExpression.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/UnaryExpression.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/UnaryExpression.java
index 2449290..ad2d144 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/UnaryExpression.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/UnaryExpression.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.expressions;
+package org.apache.calcite.linq4j.tree;
 
 import java.lang.reflect.Type;
 
@@ -30,8 +30,7 @@ public class UnaryExpression extends Expression {
     this.expression = expression;
   }
 
-  @Override
-  public Expression accept(Visitor visitor) {
+  @Override public Expression accept(Visitor visitor) {
     visitor = visitor.preVisit(this);
     Expression expression = this.expression.accept(visitor);
     return visitor.visit(this, expression);
@@ -55,8 +54,7 @@ public class UnaryExpression extends Expression {
     }
   }
 
-  @Override
-  public boolean equals(Object o) {
+  @Override public boolean equals(Object o) {
     if (this == o) {
       return true;
     }
@@ -76,8 +74,7 @@ public class UnaryExpression extends Expression {
     return true;
   }
 
-  @Override
-  public int hashCode() {
+  @Override public int hashCode() {
     int result = super.hashCode();
     result = 31 * result + expression.hashCode();
     return result;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/tree/Visitor.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/Visitor.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/Visitor.java
index 30add75..ed4f86a 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/Visitor.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/Visitor.java
@@ -14,9 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.expressions;
+package org.apache.calcite.linq4j.tree;
 
-import net.hydromatic.linq4j.Linq4j;
+import org.apache.calcite.linq4j.Linq4j;
 
 import java.util.List;
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/tree/WhileStatement.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/WhileStatement.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/WhileStatement.java
index bdf772a..b2b1eec 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/WhileStatement.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/WhileStatement.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.expressions;
+package org.apache.calcite.linq4j.tree;
 
 /**
  * Represents a "while" statement.
@@ -31,22 +31,19 @@ public class WhileStatement extends Statement {
     this.body = body;
   }
 
-  @Override
-  public Statement accept(Visitor visitor) {
+  @Override public Statement accept(Visitor visitor) {
     visitor = visitor.preVisit(this);
     final Expression condition1 = condition.accept(visitor);
     final Statement body1 = body.accept(visitor);
     return visitor.visit(this, condition1, body1);
   }
 
-  @Override
-  void accept0(ExpressionWriter writer) {
+  @Override void accept0(ExpressionWriter writer) {
     writer.append("while (").append(condition).append(") ").append(
         Blocks.toBlock(body));
   }
 
-  @Override
-  public boolean equals(Object o) {
+  @Override public boolean equals(Object o) {
     if (this == o) {
       return true;
     }
@@ -69,8 +66,7 @@ public class WhileStatement extends Statement {
     return true;
   }
 
-  @Override
-  public int hashCode() {
+  @Override public int hashCode() {
     int result = super.hashCode();
     result = 31 * result + condition.hashCode();
     result = 31 * result + body.hashCode();

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/tree/package-info.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/package-info.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/package-info.java
index 32c4e74..f8aa0e9 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/package-info.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/package-info.java
@@ -20,11 +20,11 @@
  *
  * <p>This object model is used when the linq4j system is analyzing
  * queries that have been submitted using methods on the
- * {@link net.hydromatic.linq4j.Queryable} interface. The system attempts
+ * {@link org.apache.calcite.linq4j.Queryable} interface. The system attempts
  * to understand the intent of the query and reorganize it for
  * efficiency; for example, it may attempt to push down filters to the
  * source SQL system.</p>
  */
-package net.hydromatic.linq4j.expressions;
+package org.apache.calcite.linq4j.tree;
 
 // End package-info.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/test/java/com/example/Linq4jExample.java
----------------------------------------------------------------------
diff --git a/linq4j/src/test/java/com/example/Linq4jExample.java b/linq4j/src/test/java/com/example/Linq4jExample.java
index 4dbe8e7..aa88f16 100644
--- a/linq4j/src/test/java/com/example/Linq4jExample.java
+++ b/linq4j/src/test/java/com/example/Linq4jExample.java
@@ -16,8 +16,11 @@
  */
 package com.example;
 
-import net.hydromatic.linq4j.Linq4j;
-import net.hydromatic.linq4j.function.*;
+import org.apache.calcite.linq4j.Linq4j;
+import org.apache.calcite.linq4j.function.Function0;
+import org.apache.calcite.linq4j.function.Function1;
+import org.apache.calcite.linq4j.function.Function2;
+import org.apache.calcite.linq4j.function.Functions;
 
 /**
  * Example using linq4j to query in-memory collections.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/test/java/org/apache/calcite/linq4j/function/FunctionTest.java
----------------------------------------------------------------------
diff --git a/linq4j/src/test/java/org/apache/calcite/linq4j/function/FunctionTest.java b/linq4j/src/test/java/org/apache/calcite/linq4j/function/FunctionTest.java
index af26581..9db57a5 100644
--- a/linq4j/src/test/java/org/apache/calcite/linq4j/function/FunctionTest.java
+++ b/linq4j/src/test/java/org/apache/calcite/linq4j/function/FunctionTest.java
@@ -14,12 +14,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.function;
+package org.apache.calcite.linq4j.function;
 
 import org.junit.Assert;
 import org.junit.Test;
 
-import java.util.*;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
 
 /**
  * Test for {@link Functions}.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/test/java/org/apache/calcite/linq4j/function/package-info.java
----------------------------------------------------------------------
diff --git a/linq4j/src/test/java/org/apache/calcite/linq4j/function/package-info.java b/linq4j/src/test/java/org/apache/calcite/linq4j/function/package-info.java
index 92d150f..0871e72 100644
--- a/linq4j/src/test/java/org/apache/calcite/linq4j/function/package-info.java
+++ b/linq4j/src/test/java/org/apache/calcite/linq4j/function/package-info.java
@@ -18,6 +18,6 @@
 /**
  * Tests for functions.
  */
-package net.hydromatic.linq4j.function;
+package org.apache.calcite.linq4j.function;
 
 // End package-info.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/test/java/org/apache/calcite/linq4j/test/BlockBuilderBase.java
----------------------------------------------------------------------
diff --git a/linq4j/src/test/java/org/apache/calcite/linq4j/test/BlockBuilderBase.java b/linq4j/src/test/java/org/apache/calcite/linq4j/test/BlockBuilderBase.java
index ba374b0..8fa837e 100644
--- a/linq4j/src/test/java/org/apache/calcite/linq4j/test/BlockBuilderBase.java
+++ b/linq4j/src/test/java/org/apache/calcite/linq4j/test/BlockBuilderBase.java
@@ -14,9 +14,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.test;
+package org.apache.calcite.linq4j.test;
 
-import net.hydromatic.linq4j.expressions.*;
+import org.apache.calcite.linq4j.tree.BlockBuilder;
+import org.apache.calcite.linq4j.tree.BlockStatement;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.linq4j.tree.ParameterExpression;
+import org.apache.calcite.linq4j.tree.Statement;
 
 /**
  * Base methods and constant for simplified Expression testing

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/test/java/org/apache/calcite/linq4j/test/BlockBuilderTest.java
----------------------------------------------------------------------
diff --git a/linq4j/src/test/java/org/apache/calcite/linq4j/test/BlockBuilderTest.java b/linq4j/src/test/java/org/apache/calcite/linq4j/test/BlockBuilderTest.java
index 798bc58..ce8b03b 100644
--- a/linq4j/src/test/java/org/apache/calcite/linq4j/test/BlockBuilderTest.java
+++ b/linq4j/src/test/java/org/apache/calcite/linq4j/test/BlockBuilderTest.java
@@ -14,14 +14,22 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.test;
+package org.apache.calcite.linq4j.test;
 
-import net.hydromatic.linq4j.expressions.*;
+import org.apache.calcite.linq4j.tree.BinaryExpression;
+import org.apache.calcite.linq4j.tree.BlockBuilder;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.ExpressionType;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.linq4j.tree.OptimizeVisitor;
+import org.apache.calcite.linq4j.tree.Visitor;
 
 import org.junit.Before;
 import org.junit.Test;
 
-import static net.hydromatic.linq4j.test.BlockBuilderBase.*;
+import static org.apache.calcite.linq4j.test.BlockBuilderBase.FOUR;
+import static org.apache.calcite.linq4j.test.BlockBuilderBase.ONE;
+import static org.apache.calcite.linq4j.test.BlockBuilderBase.TWO;
 
 import static org.junit.Assert.assertEquals;
 
@@ -44,11 +52,11 @@ public class BlockBuilderTest {
     b.add(nested.toBlock());
     assertEquals(
         "{\n"
-        + "  final int x = 1 + 2;\n"
-        + "  {\n"
-        + "    return x + x;\n"
-        + "  }\n"
-        + "}\n",
+            + "  final int x = 1 + 2;\n"
+            + "  {\n"
+            + "    return x + x;\n"
+            + "  }\n"
+            + "}\n",
         b.toBlock().toString());
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/test/java/org/apache/calcite/linq4j/test/DeterministicTest.java
----------------------------------------------------------------------
diff --git a/linq4j/src/test/java/org/apache/calcite/linq4j/test/DeterministicTest.java b/linq4j/src/test/java/org/apache/calcite/linq4j/test/DeterministicTest.java
index b04c9d3..aa53112 100644
--- a/linq4j/src/test/java/org/apache/calcite/linq4j/test/DeterministicTest.java
+++ b/linq4j/src/test/java/org/apache/calcite/linq4j/test/DeterministicTest.java
@@ -14,12 +14,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.test;
-
-import net.hydromatic.linq4j.expressions.*;
-
-import net.hydromatic.linq4j.function.Deterministic;
-import net.hydromatic.linq4j.function.NonDeterministic;
+package org.apache.calcite.linq4j.test;
+
+import org.apache.calcite.linq4j.function.Deterministic;
+import org.apache.calcite.linq4j.function.NonDeterministic;
+import org.apache.calcite.linq4j.tree.Blocks;
+import org.apache.calcite.linq4j.tree.ClassDeclarationFinder;
+import org.apache.calcite.linq4j.tree.DeterministicCodeOptimizer;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.linq4j.tree.ParameterExpression;
+import org.apache.calcite.linq4j.tree.Types;
 
 import org.junit.Test;
 
@@ -28,9 +33,15 @@ import java.math.BigInteger;
 import java.util.Collections;
 import java.util.concurrent.Callable;
 
-import static net.hydromatic.linq4j.test.BlockBuilderBase.*;
+import static org.apache.calcite.linq4j.test.BlockBuilderBase.FOUR;
+import static org.apache.calcite.linq4j.test.BlockBuilderBase.ONE;
+import static org.apache.calcite.linq4j.test.BlockBuilderBase.THREE;
+import static org.apache.calcite.linq4j.test.BlockBuilderBase.TWO;
+import static org.apache.calcite.linq4j.test.BlockBuilderBase.optimize;
+import static org.apache.calcite.linq4j.test.BlockBuilderBase.optimizeExpression;
 
-import static org.hamcrest.CoreMatchers.*;
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertThat;
 
 /**
@@ -130,8 +141,7 @@ public class DeterministicTest {
                     "test",
                     Collections.<ParameterExpression>emptyList(),
                     Blocks.toFunctionBlock(Expressions.add(ONE, TWO))))),
-        equalTo(
-            "{\n"
+        equalTo("{\n"
             + "  return new Runnable(){\n"
             + "      int test() {\n"
             + "        return $L4J$C$1_2;\n"
@@ -154,8 +164,7 @@ public class DeterministicTest {
                     "test",
                     Collections.<ParameterExpression>emptyList(),
                     Blocks.toFunctionBlock(Expressions.add(ONE, TWO)))))),
-        equalTo(
-            "{\n"
+        equalTo("{\n"
             + "  return new Runnable(){\n"
             + "      int test() {\n"
             + "        return $L4J$C$1_2;\n"
@@ -180,8 +189,7 @@ public class DeterministicTest {
                     Blocks.toFunctionBlock(Expressions.multiply(
                         Expressions.add(ONE, TWO),
                         Expressions.subtract(ONE, TWO)))))),
-        equalTo(
-            "{\n"
+        equalTo("{\n"
             + "  return new Runnable(){\n"
             + "      int test() {\n"
             + "        return $L4J$C$1_2_1_20;\n"
@@ -208,16 +216,16 @@ public class DeterministicTest {
                     Blocks.toFunctionBlock(
                         Expressions.multiply(Expressions.add(ONE, TWO),
                             THREE))))),
-      equalTo("{\n"
-          + "  return new Runnable(){\n"
-          + "      int test() {\n"
-          + "        return $L4J$C$1_2_3;\n"
-          + "      }\n"
-          + "\n"
-          + "      static final int $L4J$C$1_2 = 1 + 2;\n"
-          + "      static final int $L4J$C$1_2_3 = $L4J$C$1_2 * 3;\n"
-          + "    };\n"
-          + "}\n"));
+        equalTo("{\n"
+            + "  return new Runnable(){\n"
+            + "      int test() {\n"
+            + "        return $L4J$C$1_2_3;\n"
+            + "      }\n"
+            + "\n"
+            + "      static final int $L4J$C$1_2 = 1 + 2;\n"
+            + "      static final int $L4J$C$1_2_3 = $L4J$C$1_2 * 3;\n"
+            + "    };\n"
+            + "}\n"));
   }
 
   @Test public void testFactorOutNestedClasses() {
@@ -250,8 +258,7 @@ public class DeterministicTest {
                                                 THREE)))),
                                 "call",
                                 Collections.<Expression>emptyList())))))),
-        equalTo(
-            "{\n"
+        equalTo("{\n"
             + "  return new Runnable(){\n"
             + "      int test() {\n"
             + "        return $L4J$C$1_4 + new java.util.concurrent.Callable(){\n"
@@ -306,8 +313,7 @@ public class DeterministicTest {
                     Collections.<ParameterExpression>emptyList(),
                     Blocks.toFunctionBlock(
                         Expressions.typeIs(ONE, Boolean.class))))),
-        equalTo(
-            "{\n"
+        equalTo("{\n"
             + "  return new Runnable(){\n"
             + "      int test() {\n"
             + "        return 1 instanceof Boolean;\n"
@@ -381,8 +387,7 @@ public class DeterministicTest {
                                 Types.lookupMethod(BigInteger.class, "valueOf",
                                     long.class),
                                 Expressions.constant(42L))))))),
-        equalTo(
-            "{\n"
+        equalTo("{\n"
             + "  return new Runnable(){\n"
             + "      int test() {\n"
             + "        return "
@@ -447,14 +452,13 @@ public class DeterministicTest {
                     Blocks.toFunctionBlock(Expressions.call(null,
                         Types.lookupMethod(TestClass.class,
                             "deterministic", int.class), ONE))))),
-        equalTo(
-            "{\n"
+        equalTo("{\n"
             + "  return new Runnable(){\n"
             + "      int test() {\n"
-            + "        return $L4J$C$net_hydromatic_linq4j_test_DeterministicTest_TestClass_determin1da033bf;\n"
+            + "        return $L4J$C$org_apache_calcite_linq4j_test_DeterministicTest_TestClass_dete33e8af1c;\n"
             + "      }\n"
             + "\n"
-            + "      static final int $L4J$C$net_hydromatic_linq4j_test_DeterministicTest_TestClass_determin1da033bf = net.hydromatic.linq4j.test.DeterministicTest.TestClass.deterministic(1);\n"
+            + "      static final int $L4J$C$org_apache_calcite_linq4j_test_DeterministicTest_TestClass_dete33e8af1c = org.apache.calcite.linq4j.test.DeterministicTest.TestClass.deterministic(1);\n"
             + "    };\n"
             + "}\n"));
   }
@@ -473,11 +477,10 @@ public class DeterministicTest {
                     Blocks.toFunctionBlock(Expressions.call(null,
                         Types.lookupMethod(TestClass.class,
                             "nonDeterministic", int.class), ONE))))),
-        equalTo(
-            "{\n"
+        equalTo("{\n"
             + "  return new Runnable(){\n"
             + "      int test() {\n"
-            + "        return net.hydromatic.linq4j.test.DeterministicTest.TestClass.nonDeterministic(1);\n"
+            + "        return org.apache.calcite.linq4j.test.DeterministicTest.TestClass.nonDeterministic(1);\n"
             + "      }\n"
             + "\n"
             + "    };\n"
@@ -498,14 +501,13 @@ public class DeterministicTest {
                     Blocks.toFunctionBlock(Expressions.call(null,
                         Types.lookupMethod(TestDeterministicClass.class,
                             "deterministic", int.class), ONE))))),
-        equalTo(
-            "{\n"
+        equalTo("{\n"
             + "  return new Runnable(){\n"
             + "      int test() {\n"
-            + "        return $L4J$C$net_hydromatic_linq4j_test_DeterministicTest_TestDeterministicCa1bc6d17;\n"
+            + "        return $L4J$C$org_apache_calcite_linq4j_test_DeterministicTest_TestDeterminis9de610da;\n"
             + "      }\n"
             + "\n"
-            + "      static final int $L4J$C$net_hydromatic_linq4j_test_DeterministicTest_TestDeterministicCa1bc6d17 = net.hydromatic.linq4j.test.DeterministicTest.TestDeterministicClass.deterministic(1);\n"
+            + "      static final int $L4J$C$org_apache_calcite_linq4j_test_DeterministicTest_TestDeterminis9de610da = org.apache.calcite.linq4j.test.DeterministicTest.TestDeterministicClass.deterministic(1);\n"
             + "    };\n"
             + "}\n"));
   }
@@ -524,11 +526,10 @@ public class DeterministicTest {
                     Blocks.toFunctionBlock(Expressions.call(null,
                         Types.lookupMethod(TestDeterministicClass.class,
                             "nonDeterministic", int.class), ONE))))),
-        equalTo(
-            "{\n"
+        equalTo("{\n"
             + "  return new Runnable(){\n"
             + "      int test() {\n"
-            + "        return net.hydromatic.linq4j.test.DeterministicTest.TestDeterministicClass.nonDeterministic(1);\n"
+            + "        return org.apache.calcite.linq4j.test.DeterministicTest.TestDeterministicClass.nonDeterministic(1);\n"
             + "      }\n"
             + "\n"
             + "    };\n"


[47/58] [abbrv] [partial] incubator-calcite git commit: [CALCITE-306] Standardize code style for "import package.*; "

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableSemiJoin.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableSemiJoin.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableSemiJoin.java
new file mode 100644
index 0000000..fc87572
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableSemiJoin.java
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.linq4j.tree.BlockBuilder;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.InvalidRelException;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.JoinInfo;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.core.SemiJoin;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.util.BuiltInMethod;
+import org.apache.calcite.util.ImmutableIntList;
+import org.apache.calcite.util.Util;
+
+/** Implementation of {@link org.apache.calcite.rel.core.SemiJoin} in
+ * {@link org.apache.calcite.adapter.enumerable.EnumerableConvention enumerable calling convention}. */
+public class EnumerableSemiJoin extends SemiJoin implements EnumerableRel {
+  EnumerableSemiJoin(
+      RelOptCluster cluster,
+      RelTraitSet traits,
+      RelNode left,
+      RelNode right,
+      RexNode condition,
+      ImmutableIntList leftKeys,
+      ImmutableIntList rightKeys)
+      throws InvalidRelException {
+    super(cluster, traits, left, right, condition, leftKeys, rightKeys);
+  }
+
+  @Override public SemiJoin copy(RelTraitSet traitSet, RexNode condition,
+      RelNode left, RelNode right, JoinRelType joinType,
+      boolean semiJoinDone) {
+    assert joinType == JoinRelType.INNER;
+    final JoinInfo joinInfo = JoinInfo.of(left, right, condition);
+    assert joinInfo.isEqui();
+    try {
+      return new EnumerableSemiJoin(getCluster(), traitSet, left, right,
+          condition, joinInfo.leftKeys, joinInfo.rightKeys);
+    } catch (InvalidRelException e) {
+      // Semantic error not possible. Must be a bug. Convert to
+      // internal error.
+      throw new AssertionError(e);
+    }
+  }
+
+  @Override public RelOptCost computeSelfCost(RelOptPlanner planner) {
+    double rowCount = RelMetadataQuery.getRowCount(this);
+
+    // Right-hand input is the "build", and hopefully small, input.
+    final double rightRowCount = right.getRows();
+    final double leftRowCount = left.getRows();
+    if (Double.isInfinite(leftRowCount)) {
+      rowCount = leftRowCount;
+    } else {
+      rowCount += Util.nLogN(leftRowCount);
+    }
+    if (Double.isInfinite(rightRowCount)) {
+      rowCount = rightRowCount;
+    } else {
+      rowCount += rightRowCount;
+    }
+    return planner.getCostFactory().makeCost(rowCount, 0, 0).multiplyBy(.01d);
+  }
+
+  public Result implement(EnumerableRelImplementor implementor, Prefer pref) {
+    BlockBuilder builder = new BlockBuilder();
+    final Result leftResult =
+        implementor.visitChild(this, 0, (EnumerableRel) left, pref);
+    Expression leftExpression =
+        builder.append(
+            "left", leftResult.block);
+    final Result rightResult =
+        implementor.visitChild(this, 1, (EnumerableRel) right, pref);
+    Expression rightExpression =
+        builder.append(
+            "right", rightResult.block);
+    final PhysType physType = leftResult.physType;
+    return implementor.result(
+        physType,
+        builder.append(
+            Expressions.call(
+                BuiltInMethod.SEMI_JOIN.method,
+                Expressions.list(
+                    leftExpression,
+                    rightExpression,
+                    leftResult.physType.generateAccessor(leftKeys),
+                    rightResult.physType.generateAccessor(rightKeys))))
+            .toBlock());
+  }
+}
+
+// End EnumerableSemiJoin.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableSemiJoinRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableSemiJoinRule.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableSemiJoinRule.java
new file mode 100644
index 0000000..f078e7a
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableSemiJoinRule.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.rel.InvalidRelException;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.core.SemiJoin;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/** Planner rule that converts a
+ * {@link org.apache.calcite.rel.core.SemiJoin} relational expression
+ * {@link org.apache.calcite.adapter.enumerable.EnumerableConvention enumerable calling convention}. */
+class EnumerableSemiJoinRule extends ConverterRule {
+  EnumerableSemiJoinRule() {
+    super(SemiJoin.class, Convention.NONE, EnumerableConvention.INSTANCE,
+        "EnumerableSemiJoinRule");
+  }
+
+  @Override public RelNode convert(RelNode rel) {
+    final SemiJoin semiJoin = (SemiJoin) rel;
+    List<RelNode> newInputs = new ArrayList<RelNode>();
+    for (RelNode input : semiJoin.getInputs()) {
+      if (!(input.getConvention() instanceof EnumerableConvention)) {
+        input =
+            convert(input,
+                input.getTraitSet().replace(EnumerableConvention.INSTANCE));
+      }
+      newInputs.add(input);
+    }
+    try {
+      return new EnumerableSemiJoin(
+          semiJoin.getCluster(),
+          semiJoin.getTraitSet().replace(EnumerableConvention.INSTANCE),
+          newInputs.get(0),
+          newInputs.get(1),
+          semiJoin.getCondition(),
+          semiJoin.leftKeys,
+          semiJoin.rightKeys);
+    } catch (InvalidRelException e) {
+      EnumerableRules.LOGGER.fine(e.toString());
+      return null;
+    }
+  }
+}
+
+// End EnumerableSemiJoinRule.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableSort.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableSort.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableSort.java
new file mode 100644
index 0000000..7c6bec5
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableSort.java
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.linq4j.tree.BlockBuilder;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.util.BuiltInMethod;
+import org.apache.calcite.util.Pair;
+
+/** Implementation of {@link org.apache.calcite.rel.core.Sort} in
+ * {@link org.apache.calcite.adapter.enumerable.EnumerableConvention enumerable calling convention}. */
+public class EnumerableSort extends Sort implements EnumerableRel {
+  public EnumerableSort(RelOptCluster cluster, RelTraitSet traitSet,
+      RelNode child, RelCollation collation, RexNode offset, RexNode fetch) {
+    super(cluster, traitSet, child, collation, offset, fetch);
+    assert getConvention() instanceof EnumerableConvention;
+    assert getConvention() == child.getConvention();
+  }
+
+  @Override public EnumerableSort copy(
+      RelTraitSet traitSet,
+      RelNode newInput,
+      RelCollation newCollation,
+      RexNode offset,
+      RexNode fetch) {
+    return new EnumerableSort(getCluster(), traitSet, newInput, newCollation,
+        offset, fetch);
+  }
+
+  public Result implement(EnumerableRelImplementor implementor, Prefer pref) {
+    final BlockBuilder builder = new BlockBuilder();
+    final EnumerableRel child = (EnumerableRel) getInput();
+    final Result result = implementor.visitChild(this, 0, child, pref);
+    final PhysType physType =
+        PhysTypeImpl.of(
+            implementor.getTypeFactory(),
+            getRowType(),
+            result.format);
+    Expression childExp =
+        builder.append("child", result.block);
+
+    PhysType inputPhysType = result.physType;
+    final Pair<Expression, Expression> pair =
+        inputPhysType.generateCollationKey(
+            collation.getFieldCollations());
+
+    builder.add(
+        Expressions.return_(null,
+            Expressions.call(childExp,
+                BuiltInMethod.ORDER_BY.method,
+                Expressions.list(
+                    builder.append("keySelector", pair.left))
+                    .appendIfNotNull(builder.appendIfNotNull("comparator",
+                        pair.right)))));
+    return implementor.result(physType, builder.toBlock());
+  }
+}
+
+// End EnumerableSort.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableSortRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableSortRule.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableSortRule.java
new file mode 100644
index 0000000..35a7e33
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableSortRule.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.core.Sort;
+
+/**
+ * Rule to convert an {@link org.apache.calcite.rel.core.Sort} to an
+ * {@link EnumerableSort}.
+ */
+class EnumerableSortRule extends ConverterRule {
+  EnumerableSortRule() {
+    super(Sort.class, Convention.NONE, EnumerableConvention.INSTANCE,
+        "EnumerableSortRule");
+  }
+
+  public RelNode convert(RelNode rel) {
+    final Sort sort = (Sort) rel;
+    if (sort.offset != null || sort.fetch != null) {
+      return null;
+    }
+    final RelTraitSet traitSet =
+        sort.getTraitSet().replace(EnumerableConvention.INSTANCE);
+    final RelNode input = sort.getInput();
+    return new EnumerableSort(
+        rel.getCluster(),
+        traitSet,
+        convert(
+            input,
+            input.getTraitSet().replace(EnumerableConvention.INSTANCE)),
+        sort.getCollation(),
+        null,
+        null);
+  }
+}
+
+// End EnumerableSortRule.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableTableFunctionScan.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableTableFunctionScan.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableTableFunctionScan.java
new file mode 100644
index 0000000..9ed359f
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableTableFunctionScan.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.linq4j.tree.BlockBuilder;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.TableFunctionScan;
+import org.apache.calcite.rel.metadata.RelColumnMapping;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexNode;
+
+import java.lang.reflect.Type;
+import java.util.List;
+import java.util.Set;
+
+/** Implementation of {@link org.apache.calcite.rel.core.TableFunctionScan} in
+ * {@link org.apache.calcite.adapter.enumerable.EnumerableConvention enumerable calling convention}. */
+public class EnumerableTableFunctionScan extends TableFunctionScan
+    implements EnumerableRel {
+
+  public EnumerableTableFunctionScan(RelOptCluster cluster,
+      RelTraitSet traits, List<RelNode> inputs, Type elementType,
+      RelDataType rowType, RexNode call,
+      Set<RelColumnMapping> columnMappings) {
+    super(cluster, traits, inputs, call, elementType, rowType,
+      columnMappings);
+  }
+
+  @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+    return new EnumerableTableFunctionScan(getCluster(), traitSet, inputs,
+        getElementType(), getRowType(), getCall(), getColumnMappings());
+  }
+
+  public Result implement(EnumerableRelImplementor implementor, Prefer pref) {
+    BlockBuilder bb = new BlockBuilder();
+     // Non-array user-specified types are not supported yet
+    final PhysType physType =
+        PhysTypeImpl.of(
+            implementor.getTypeFactory(),
+            getRowType(),
+            getElementType() == null /* e.g. not known */
+            || (getElementType() instanceof Class
+                && Object[].class.isAssignableFrom((Class) getElementType()))
+            ? JavaRowFormat.ARRAY
+            : JavaRowFormat.CUSTOM);
+    RexToLixTranslator t = RexToLixTranslator.forAggregation(
+        (JavaTypeFactory) getCluster().getTypeFactory(), bb, null);
+    final Expression translated = t.translate(getCall());
+    bb.add(Expressions.return_(null, translated));
+    return implementor.result(physType, bb.toBlock());
+  }
+}
+
+// End EnumerableTableFunctionScan.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableTableFunctionScanRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableTableFunctionScanRule.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableTableFunctionScanRule.java
new file mode 100644
index 0000000..b22c528
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableTableFunctionScanRule.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.logical.LogicalTableFunctionScan;
+
+/** Planner rule that converts a
+ * {@link org.apache.calcite.rel.logical.LogicalTableFunctionScan}
+ * relational expression
+ * {@link org.apache.calcite.adapter.enumerable.EnumerableConvention enumerable calling convention}. */
+public class EnumerableTableFunctionScanRule extends ConverterRule {
+  public EnumerableTableFunctionScanRule() {
+    super(LogicalTableFunctionScan.class, Convention.NONE,
+        EnumerableConvention.INSTANCE, "EnumerableTableFunctionRule");
+  }
+
+  @Override public RelNode convert(RelNode rel) {
+    final RelTraitSet traitSet =
+        rel.getTraitSet().replace(EnumerableConvention.INSTANCE);
+    LogicalTableFunctionScan tbl = (LogicalTableFunctionScan) rel;
+    return new EnumerableTableFunctionScan(rel.getCluster(), traitSet,
+        tbl.getInputs(), tbl.getElementType(), tbl.getRowType(),
+        tbl.getCall(), tbl.getColumnMappings());
+  }
+}
+
+// End EnumerableTableFunctionScanRule.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableTableModify.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableTableModify.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableTableModify.java
new file mode 100644
index 0000000..944577c
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableTableModify.java
@@ -0,0 +1,149 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.linq4j.tree.BlockBuilder;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.linq4j.tree.ParameterExpression;
+import org.apache.calcite.linq4j.tree.Types;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.prepare.Prepare;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.TableModify;
+import org.apache.calcite.schema.ModifiableTable;
+import org.apache.calcite.util.BuiltInMethod;
+
+import java.lang.reflect.Modifier;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+/** Implementation of {@link org.apache.calcite.rel.core.TableModify} in
+ * {@link org.apache.calcite.adapter.enumerable.EnumerableConvention enumerable calling convention}. */
+public class EnumerableTableModify extends TableModify
+    implements EnumerableRel {
+  public EnumerableTableModify(RelOptCluster cluster, RelTraitSet traits,
+      RelOptTable table, Prepare.CatalogReader catalogReader, RelNode child,
+      Operation operation, List<String> updateColumnList, boolean flattened) {
+    super(cluster, traits, table, catalogReader, child, operation,
+        updateColumnList, flattened);
+    assert child.getConvention() instanceof EnumerableConvention;
+    assert getConvention() instanceof EnumerableConvention;
+    final ModifiableTable modifiableTable =
+        table.unwrap(ModifiableTable.class);
+    if (modifiableTable == null) {
+      throw new AssertionError(); // TODO: user error in validator
+    }
+  }
+
+  @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+    return new EnumerableTableModify(
+        getCluster(),
+        traitSet,
+        getTable(),
+        getCatalogReader(),
+        sole(inputs),
+        getOperation(),
+        getUpdateColumnList(),
+        isFlattened());
+  }
+
+  public Result implement(EnumerableRelImplementor implementor, Prefer pref) {
+    final BlockBuilder builder = new BlockBuilder();
+    final Result result = implementor.visitChild(
+        this, 0, (EnumerableRel) getInput(), pref);
+    Expression childExp =
+        builder.append(
+            "child", result.block);
+    final ParameterExpression collectionParameter =
+        Expressions.parameter(Collection.class,
+            builder.newName("collection"));
+    final Expression expression = table.getExpression(ModifiableTable.class);
+    assert expression != null; // TODO: user error in validator
+    assert ModifiableTable.class.isAssignableFrom(
+        Types.toClass(expression.getType())) : expression.getType();
+    builder.add(
+        Expressions.declare(
+            Modifier.FINAL,
+            collectionParameter,
+            Expressions.call(
+                expression,
+                BuiltInMethod.MODIFIABLE_TABLE_GET_MODIFIABLE_COLLECTION
+                    .method)));
+    final Expression countParameter =
+        builder.append(
+            "count",
+            Expressions.call(collectionParameter, "size"),
+            false);
+    Expression convertedChildExp;
+    if (!getInput().getRowType().equals(getRowType())) {
+      final JavaTypeFactory typeFactory =
+          (JavaTypeFactory) getCluster().getTypeFactory();
+      PhysType physType =
+          PhysTypeImpl.of(typeFactory, table.getRowType(),
+              JavaRowFormat.CUSTOM);
+      List<Expression> expressionList = new ArrayList<Expression>();
+      final PhysType childPhysType = result.physType;
+      final ParameterExpression o_ =
+          Expressions.parameter(childPhysType.getJavaRowType(), "o");
+      final int fieldCount =
+          childPhysType.getRowType().getFieldCount();
+      for (int i = 0; i < fieldCount; i++) {
+        expressionList.add(childPhysType.fieldReference(o_, i,
+            physType.getJavaFieldType(i)));
+      }
+      convertedChildExp =
+          builder.append(
+              "convertedChild",
+              Expressions.call(
+                  childExp,
+                  BuiltInMethod.SELECT.method,
+                  Expressions.lambda(
+                      physType.record(expressionList), o_)));
+    } else {
+      convertedChildExp = childExp;
+    }
+    builder.add(
+        Expressions.statement(
+            Expressions.call(
+                convertedChildExp, "into", collectionParameter)));
+    builder.add(
+        Expressions.return_(
+            null,
+            Expressions.call(
+                BuiltInMethod.SINGLETON_ENUMERABLE.method,
+                Expressions.convert_(
+                    Expressions.subtract(
+                        Expressions.call(
+                            collectionParameter, "size"),
+                        countParameter),
+                    long.class))));
+    final PhysType physType =
+        PhysTypeImpl.of(
+            implementor.getTypeFactory(),
+            getRowType(),
+            pref == Prefer.ARRAY
+                ? JavaRowFormat.ARRAY : JavaRowFormat.SCALAR);
+    return implementor.result(physType, builder.toBlock());
+  }
+}
+
+// End EnumerableTableModify.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableTableModifyRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableTableModifyRule.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableTableModifyRule.java
new file mode 100644
index 0000000..11c6f2c
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableTableModifyRule.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.logical.LogicalTableModify;
+import org.apache.calcite.schema.ModifiableTable;
+
+/** Planner rule that converts a
+ * {@link org.apache.calcite.rel.logical.LogicalTableModify}
+ * relational expression
+ * {@link org.apache.calcite.adapter.enumerable.EnumerableConvention enumerable calling convention}. */
+public class EnumerableTableModifyRule extends ConverterRule {
+  EnumerableTableModifyRule() {
+    super(LogicalTableModify.class, Convention.NONE,
+        EnumerableConvention.INSTANCE, "EnumerableTableModificationRule");
+  }
+
+  @Override public RelNode convert(RelNode rel) {
+    final LogicalTableModify modify =
+        (LogicalTableModify) rel;
+    final ModifiableTable modifiableTable =
+        modify.getTable().unwrap(ModifiableTable.class);
+    if (modifiableTable == null) {
+      return null;
+    }
+    final RelTraitSet traitSet =
+        modify.getTraitSet().replace(EnumerableConvention.INSTANCE);
+    return new EnumerableTableModify(
+        modify.getCluster(), traitSet,
+        modify.getTable(),
+        modify.getCatalogReader(),
+        convert(modify.getInput(), traitSet),
+        modify.getOperation(),
+        modify.getUpdateColumnList(),
+        modify.isFlattened());
+  }
+}
+
+// End EnumerableTableModifyRule.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableTableScan.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableTableScan.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableTableScan.java
new file mode 100644
index 0000000..7eb30f1
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableTableScan.java
@@ -0,0 +1,153 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.interpreter.Row;
+import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
+import org.apache.calcite.linq4j.Enumerable;
+import org.apache.calcite.linq4j.Queryable;
+import org.apache.calcite.linq4j.function.Function1;
+import org.apache.calcite.linq4j.tree.Blocks;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.linq4j.tree.ParameterExpression;
+import org.apache.calcite.linq4j.tree.Types;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.util.BuiltInMethod;
+
+import java.lang.reflect.Type;
+import java.util.ArrayList;
+import java.util.List;
+
+/** Implementation of {@link org.apache.calcite.rel.core.TableScan} in
+ * {@link org.apache.calcite.adapter.enumerable.EnumerableConvention enumerable calling convention}. */
+public class EnumerableTableScan
+    extends TableScan
+    implements EnumerableRel {
+  private final Class elementType;
+
+  public EnumerableTableScan(RelOptCluster cluster, RelTraitSet traitSet,
+      RelOptTable table, Class elementType) {
+    super(cluster, traitSet, table);
+    assert getConvention() instanceof EnumerableConvention;
+    this.elementType = elementType;
+  }
+
+  private Expression getExpression(PhysType physType) {
+    final Expression expression = table.getExpression(Queryable.class);
+    final Expression expression2 = toEnumerable(expression);
+    assert Types.isAssignableFrom(Enumerable.class, expression2.getType());
+    Expression expression3 = toRows(physType, expression2);
+    return expression3;
+  }
+
+  private Expression toEnumerable(Expression expression) {
+    final Type type = expression.getType();
+    if (Types.isArray(type)) {
+      if (Types.toClass(type).getComponentType().isPrimitive()) {
+        expression =
+            Expressions.call(BuiltInMethod.AS_LIST.method, expression);
+      }
+      return Expressions.call(BuiltInMethod.AS_ENUMERABLE.method, expression);
+    } else if (Types.isAssignableFrom(Iterable.class, type)
+        && !Types.isAssignableFrom(Enumerable.class, type)) {
+      return Expressions.call(BuiltInMethod.AS_ENUMERABLE2.method,
+          expression);
+    } else if (Types.isAssignableFrom(Queryable.class, type)) {
+      // Queryable extends Enumerable, but it's too "clever", so we call
+      // Queryable.asEnumerable so that operations such as take(int) will be
+      // evaluated directly.
+      return Expressions.call(expression,
+          BuiltInMethod.QUERYABLE_AS_ENUMERABLE.method);
+    }
+    return expression;
+  }
+
+  private Expression toRows(PhysType physType, Expression expression) {
+    final ParameterExpression row_ =
+        Expressions.parameter(elementType, "row");
+    List<Expression> expressionList = new ArrayList<Expression>();
+    final int fieldCount = table.getRowType().getFieldCount();
+    if (elementType == Row.class) {
+      // Convert Enumerable<Row> to Enumerable<SyntheticRecord>
+      for (int i = 0; i < fieldCount; i++) {
+        expressionList.add(
+            RexToLixTranslator.convert(
+                Expressions.call(row_,
+                    BuiltInMethod.ROW_VALUE.method,
+                    Expressions.constant(i)),
+                physType.getJavaFieldType(i)));
+      }
+    } else if (elementType == Object[].class
+        && rowType.getFieldCount() == 1) {
+      // Convert Enumerable<Object[]> to Enumerable<SyntheticRecord>
+      for (int i = 0; i < fieldCount; i++) {
+        expressionList.add(
+            RexToLixTranslator.convert(
+                Expressions.arrayIndex(row_, Expressions.constant(i)),
+                physType.getJavaFieldType(i)));
+      }
+    } else if (elementType == Object.class) {
+      if (!(physType.getJavaRowType()
+          instanceof JavaTypeFactoryImpl.SyntheticRecordType)) {
+        return expression;
+      }
+      expressionList.add(
+          RexToLixTranslator.convert(row_, physType.getJavaFieldType(0)));
+    } else {
+      return expression;
+    }
+    return Expressions.call(expression,
+        BuiltInMethod.SELECT.method,
+        Expressions.lambda(Function1.class, physType.record(expressionList),
+            row_));
+  }
+
+  private JavaRowFormat format() {
+    if (Object[].class.isAssignableFrom(elementType)) {
+      return JavaRowFormat.ARRAY;
+    } else {
+      return JavaRowFormat.CUSTOM;
+    }
+  }
+
+  @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+    return new EnumerableTableScan(getCluster(), traitSet, table,
+        elementType);
+  }
+
+  public Result implement(EnumerableRelImplementor implementor, Prefer pref) {
+    // Note that representation is ARRAY. This assumes that the table
+    // returns a Object[] for each record. Actually a Table<T> can
+    // return any type T. And, if it is a JdbcTable, we'd like to be
+    // able to generate alternate accessors that return e.g. synthetic
+    // records {T0 f0; T1 f1; ...} and don't box every primitive value.
+    final PhysType physType =
+        PhysTypeImpl.of(
+            implementor.getTypeFactory(),
+            getRowType(),
+            format());
+    final Expression expression = getExpression(physType);
+    return implementor.result(physType, Blocks.toBlock(expression));
+  }
+}
+
+// End EnumerableTableScan.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableUncollect.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableUncollect.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableUncollect.java
new file mode 100644
index 0000000..7160823
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableUncollect.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.linq4j.tree.BlockBuilder;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Uncollect;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.util.BuiltInMethod;
+
+/** Implementation of {@link org.apache.calcite.rel.core.Uncollect} in
+ * {@link org.apache.calcite.adapter.enumerable.EnumerableConvention enumerable calling convention}. */
+public class EnumerableUncollect extends Uncollect implements EnumerableRel {
+  public EnumerableUncollect(RelOptCluster cluster, RelTraitSet traitSet,
+      RelNode child) {
+    super(cluster, traitSet, child);
+    assert getConvention() instanceof EnumerableConvention;
+    assert getConvention() == child.getConvention();
+  }
+
+  @Override public EnumerableUncollect copy(RelTraitSet traitSet,
+      RelNode newInput) {
+    return new EnumerableUncollect(getCluster(), traitSet, newInput);
+  }
+
+  public Result implement(EnumerableRelImplementor implementor, Prefer pref) {
+    final BlockBuilder builder = new BlockBuilder();
+    final EnumerableRel child = (EnumerableRel) getInput();
+    final Result result = implementor.visitChild(this, 0, child, pref);
+    final PhysType physType =
+        PhysTypeImpl.of(
+            implementor.getTypeFactory(),
+            getRowType(),
+            result.format);
+
+    final JavaTypeFactory typeFactory = implementor.getTypeFactory();
+    RelDataType inputRowType = child.getRowType();
+
+    // final Enumerable<List<Employee>> child = <<child adapter>>;
+    // return child.selectMany(LIST_TO_ENUMERABLE);
+    final Expression child_ =
+        builder.append(
+            "child", result.block);
+    builder.add(
+        Expressions.return_(null,
+            Expressions.call(child_,
+                BuiltInMethod.SELECT_MANY.method,
+                Expressions.call(BuiltInMethod.LIST_TO_ENUMERABLE.method))));
+    return implementor.result(physType, builder.toBlock());
+  }
+}
+
+// End EnumerableUncollect.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableUncollectRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableUncollectRule.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableUncollectRule.java
new file mode 100644
index 0000000..797bbc2
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableUncollectRule.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.core.Uncollect;
+
+/**
+ * Rule to convert an {@link org.apache.calcite.rel.core.Uncollect} to an
+ * {@link EnumerableUncollect}.
+ */
+class EnumerableUncollectRule extends ConverterRule {
+  EnumerableUncollectRule() {
+    super(Uncollect.class, Convention.NONE, EnumerableConvention.INSTANCE,
+        "EnumerableUncollectRule");
+  }
+
+  public RelNode convert(RelNode rel) {
+    final Uncollect uncollect = (Uncollect) rel;
+    final RelTraitSet traitSet =
+        uncollect.getTraitSet().replace(EnumerableConvention.INSTANCE);
+    final RelNode input = uncollect.getInput();
+    return new EnumerableUncollect(
+        rel.getCluster(),
+        traitSet,
+        convert(input,
+            input.getTraitSet().replace(EnumerableConvention.INSTANCE)));
+  }
+}
+
+// End EnumerableUncollectRule.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableUnion.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableUnion.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableUnion.java
new file mode 100644
index 0000000..5162701
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableUnion.java
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.linq4j.tree.BlockBuilder;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Union;
+import org.apache.calcite.util.BuiltInMethod;
+
+import java.util.List;
+
+/** Implementation of {@link org.apache.calcite.rel.core.Union} in
+ * {@link org.apache.calcite.adapter.enumerable.EnumerableConvention enumerable calling convention}. */
+public class EnumerableUnion extends Union implements EnumerableRel {
+  public EnumerableUnion(RelOptCluster cluster, RelTraitSet traitSet,
+      List<RelNode> inputs, boolean all) {
+    super(cluster, traitSet, inputs, all);
+  }
+
+  public EnumerableUnion copy(RelTraitSet traitSet, List<RelNode> inputs,
+      boolean all) {
+    return new EnumerableUnion(getCluster(), traitSet, inputs, all);
+  }
+
+  public Result implement(EnumerableRelImplementor implementor, Prefer pref) {
+    final BlockBuilder builder = new BlockBuilder();
+    Expression unionExp = null;
+    for (Ord<RelNode> ord : Ord.zip(inputs)) {
+      EnumerableRel input = (EnumerableRel) ord.e;
+      final Result result = implementor.visitChild(this, ord.i, input, pref);
+      Expression childExp =
+          builder.append(
+              "child" + ord.i,
+              result.block);
+
+      if (unionExp == null) {
+        unionExp = childExp;
+      } else {
+        unionExp =
+            Expressions.call(
+                unionExp,
+                all
+                    ? BuiltInMethod.CONCAT.method
+                    : BuiltInMethod.UNION.method,
+                childExp);
+      }
+
+      // Once the first input has chosen its format, ask for the same for
+      // other inputs.
+      pref = pref.of(result.format);
+    }
+
+    builder.add(unionExp);
+    final PhysType physType =
+        PhysTypeImpl.of(
+            implementor.getTypeFactory(),
+            getRowType(),
+            pref.prefer(JavaRowFormat.CUSTOM));
+    return implementor.result(physType, builder.toBlock());
+  }
+}
+
+// End EnumerableUnion.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableUnionRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableUnionRule.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableUnionRule.java
new file mode 100644
index 0000000..42a97f1
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableUnionRule.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.logical.LogicalUnion;
+
+/**
+ * Rule to convert an {@link org.apache.calcite.rel.logical.LogicalUnion} to an
+ * {@link EnumerableUnion}.
+ */
+class EnumerableUnionRule extends ConverterRule {
+  EnumerableUnionRule() {
+    super(LogicalUnion.class, Convention.NONE, EnumerableConvention.INSTANCE,
+        "EnumerableUnionRule");
+  }
+
+  public RelNode convert(RelNode rel) {
+    final LogicalUnion union = (LogicalUnion) rel;
+    final EnumerableConvention out = EnumerableConvention.INSTANCE;
+    final RelTraitSet traitSet = union.getTraitSet().replace(out);
+    return new EnumerableUnion(rel.getCluster(), traitSet,
+        convertList(union.getInputs(), out), union.all);
+  }
+}
+
+// End EnumerableUnionRule.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableValues.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableValues.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableValues.java
new file mode 100644
index 0000000..36504b1
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableValues.java
@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.linq4j.tree.BlockBuilder;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.linq4j.tree.Primitive;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Values;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.util.BuiltInMethod;
+import org.apache.calcite.util.Pair;
+
+import java.lang.reflect.Type;
+import java.util.ArrayList;
+import java.util.List;
+
+/** Implementation of {@link org.apache.calcite.rel.core.Values} in
+ * {@link org.apache.calcite.adapter.enumerable.EnumerableConvention enumerable calling convention}. */
+public class EnumerableValues extends Values implements EnumerableRel {
+  EnumerableValues(RelOptCluster cluster, RelDataType rowType,
+      List<List<RexLiteral>> tuples, RelTraitSet traitSet) {
+    super(cluster, rowType, tuples, traitSet);
+  }
+
+  @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+    assert inputs.isEmpty();
+    return new EnumerableValues(
+        getCluster(), rowType, tuples, traitSet);
+  }
+
+  public Result implement(EnumerableRelImplementor implementor, Prefer pref) {
+/*
+          return Linq4j.asEnumerable(
+              new Object[][] {
+                  new Object[] {1, 2},
+                  new Object[] {3, 4}
+              });
+*/
+    final JavaTypeFactory typeFactory =
+        (JavaTypeFactory) getCluster().getTypeFactory();
+    final BlockBuilder builder = new BlockBuilder();
+    final PhysType physType =
+        PhysTypeImpl.of(
+            implementor.getTypeFactory(),
+            getRowType(),
+            pref.preferCustom());
+    final Type rowClass = physType.getJavaRowType();
+
+    final List<Expression> expressions = new ArrayList<Expression>();
+    final List<RelDataTypeField> fields = rowType.getFieldList();
+    for (List<RexLiteral> tuple : tuples) {
+      final List<Expression> literals = new ArrayList<Expression>();
+      for (Pair<RelDataTypeField, RexLiteral> pair
+          : Pair.zip(fields, tuple)) {
+        literals.add(
+            RexToLixTranslator.translateLiteral(
+                pair.right,
+                pair.left.getType(),
+                typeFactory,
+                RexImpTable.NullAs.NULL));
+      }
+      expressions.add(physType.record(literals));
+    }
+    builder.add(
+        Expressions.return_(
+            null,
+            Expressions.call(
+                BuiltInMethod.AS_ENUMERABLE.method,
+                Expressions.newArrayInit(
+                    Primitive.box(rowClass), expressions))));
+    return implementor.result(physType, builder.toBlock());
+  }
+}
+
+// End EnumerableValues.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableValuesRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableValuesRule.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableValuesRule.java
new file mode 100644
index 0000000..ed1b3ca
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableValuesRule.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.logical.LogicalValues;
+
+/** Planner rule that converts a
+ * {@link org.apache.calcite.rel.logical.LogicalValues}
+ * relational expression
+ * {@link org.apache.calcite.adapter.enumerable.EnumerableConvention enumerable calling convention}. */
+public class EnumerableValuesRule extends ConverterRule {
+  EnumerableValuesRule() {
+    super(LogicalValues.class, Convention.NONE, EnumerableConvention.INSTANCE,
+        "EnumerableValuesRule");
+  }
+
+  @Override public RelNode convert(RelNode rel) {
+    LogicalValues values = (LogicalValues) rel;
+    return new EnumerableValues(
+        values.getCluster(),
+        values.getRowType(),
+        values.getTuples(),
+        values.getTraitSet().replace(EnumerableConvention.INSTANCE));
+  }
+}
+
+// End EnumerableValuesRule.java


[43/58] [abbrv] [partial] incubator-calcite git commit: [CALCITE-306] Standardize code style for "import package.*; "

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/interpreter/ScanNode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/interpreter/ScanNode.java b/core/src/main/java/org/apache/calcite/interpreter/ScanNode.java
index 778b473..e2bdfd4 100644
--- a/core/src/main/java/org/apache/calcite/interpreter/ScanNode.java
+++ b/core/src/main/java/org/apache/calcite/interpreter/ScanNode.java
@@ -14,27 +14,25 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl.interpreter;
+package org.apache.calcite.interpreter;
 
-import net.hydromatic.linq4j.Enumerable;
-import net.hydromatic.linq4j.Enumerator;
-import net.hydromatic.linq4j.Queryable;
-import net.hydromatic.linq4j.function.Function1;
-
-import net.hydromatic.optiq.DataContext;
-import net.hydromatic.optiq.FilterableTable;
-import net.hydromatic.optiq.ProjectableFilterableTable;
-import net.hydromatic.optiq.QueryableTable;
-import net.hydromatic.optiq.ScannableTable;
-import net.hydromatic.optiq.SchemaPlus;
-import net.hydromatic.optiq.Schemas;
-import net.hydromatic.optiq.runtime.Enumerables;
-
-import org.eigenbase.rel.TableAccessRelBase;
-import org.eigenbase.relopt.RelOptTable;
-import org.eigenbase.rex.RexNode;
-import org.eigenbase.util.ImmutableIntList;
-import org.eigenbase.util.Util;
+import org.apache.calcite.DataContext;
+import org.apache.calcite.linq4j.Enumerable;
+import org.apache.calcite.linq4j.Enumerator;
+import org.apache.calcite.linq4j.Queryable;
+import org.apache.calcite.linq4j.function.Function1;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.runtime.Enumerables;
+import org.apache.calcite.schema.FilterableTable;
+import org.apache.calcite.schema.ProjectableFilterableTable;
+import org.apache.calcite.schema.QueryableTable;
+import org.apache.calcite.schema.ScannableTable;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.schema.Schemas;
+import org.apache.calcite.util.ImmutableIntList;
+import org.apache.calcite.util.Util;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
@@ -47,16 +45,16 @@ import java.util.List;
 
 /**
  * Interpreter node that implements a
- * {@link org.eigenbase.rel.TableAccessRelBase}.
+ * {@link org.apache.calcite.rel.core.TableScan}.
  */
 public class ScanNode implements Node {
   private final Sink sink;
-  private final TableAccessRelBase rel;
+  private final TableScan rel;
   private final ImmutableList<RexNode> filters;
   private final DataContext root;
   private final int[] projects;
 
-  public ScanNode(Interpreter interpreter, TableAccessRelBase rel,
+  public ScanNode(Interpreter interpreter, TableScan rel,
       ImmutableList<RexNode> filters, ImmutableIntList projects) {
     this.rel = rel;
     this.filters = Preconditions.checkNotNull(filters);
@@ -153,8 +151,7 @@ public class ScanNode implements Node {
     if (scannableTable != null) {
       return Enumerables.toRow(scannableTable.scan(root));
     }
-    throw new AssertionError("cannot convert table " + table
-        + " to iterable");
+    throw new AssertionError("cannot convert table " + table + " to iterable");
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/interpreter/Sink.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/interpreter/Sink.java b/core/src/main/java/org/apache/calcite/interpreter/Sink.java
index 6a07c1d..adcbac7 100644
--- a/core/src/main/java/org/apache/calcite/interpreter/Sink.java
+++ b/core/src/main/java/org/apache/calcite/interpreter/Sink.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl.interpreter;
+package org.apache.calcite.interpreter;
 
 /**
  * Sink to which to send rows.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/interpreter/SortNode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/interpreter/SortNode.java b/core/src/main/java/org/apache/calcite/interpreter/SortNode.java
index 7f0c8a6..e5fb91d 100644
--- a/core/src/main/java/org/apache/calcite/interpreter/SortNode.java
+++ b/core/src/main/java/org/apache/calcite/interpreter/SortNode.java
@@ -14,11 +14,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl.interpreter;
+package org.apache.calcite.interpreter;
 
-import org.eigenbase.rel.RelFieldCollation;
-import org.eigenbase.rel.SortRel;
-import org.eigenbase.rex.RexLiteral;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rex.RexLiteral;
 
 import com.google.common.base.Function;
 import com.google.common.collect.Iterables;
@@ -32,14 +32,14 @@ import java.util.List;
 
 /**
  * Interpreter node that implements a
- * {@link org.eigenbase.rel.SortRel}.
+ * {@link org.apache.calcite.rel.core.Sort}.
  */
 public class SortNode implements Node {
   private final Source source;
   private final Sink sink;
-  private final SortRel rel;
+  private final Sort rel;
 
-  public SortNode(Interpreter interpreter, SortRel rel) {
+  public SortNode(Interpreter interpreter, Sort rel) {
     this.rel = rel;
     this.source = interpreter.source(rel, 0);
     this.sink = interpreter.sink(rel);
@@ -129,4 +129,4 @@ public class SortNode implements Node {
   }
 }
 
-// End ScanNode.java
+// End SortNode.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/interpreter/Source.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/interpreter/Source.java b/core/src/main/java/org/apache/calcite/interpreter/Source.java
index 6f67d32..f020343 100644
--- a/core/src/main/java/org/apache/calcite/interpreter/Source.java
+++ b/core/src/main/java/org/apache/calcite/interpreter/Source.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl.interpreter;
+package org.apache.calcite.interpreter;
 
 /**
  * Source of rows.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/interpreter/ValuesNode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/interpreter/ValuesNode.java b/core/src/main/java/org/apache/calcite/interpreter/ValuesNode.java
index e60e797..fe68790 100644
--- a/core/src/main/java/org/apache/calcite/interpreter/ValuesNode.java
+++ b/core/src/main/java/org/apache/calcite/interpreter/ValuesNode.java
@@ -14,22 +14,23 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl.interpreter;
+package org.apache.calcite.interpreter;
 
-import org.eigenbase.rel.ValuesRelBase;
-import org.eigenbase.rex.RexLiteral;
+import org.apache.calcite.rel.core.Values;
+import org.apache.calcite.rex.RexLiteral;
 
 import java.util.List;
 
 /**
- * Interpreter node that implements a {@link ValuesRelBase}.
+ * Interpreter node that implements a
+ * {@link org.apache.calcite.rel.core.Values}.
  */
 public class ValuesNode implements Node {
   private final Sink sink;
-  private final ValuesRelBase rel;
+  private final Values rel;
   private final int fieldCount;
 
-  public ValuesNode(Interpreter interpreter, ValuesRelBase rel) {
+  public ValuesNode(Interpreter interpreter, Values rel) {
     this.rel = rel;
     this.sink = interpreter.sink(rel);
     this.fieldCount = rel.getRowType().getFieldCount();

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/interpreter/package-info.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/interpreter/package-info.java b/core/src/main/java/org/apache/calcite/interpreter/package-info.java
index 712d56e..a0117ce 100644
--- a/core/src/main/java/org/apache/calcite/interpreter/package-info.java
+++ b/core/src/main/java/org/apache/calcite/interpreter/package-info.java
@@ -22,6 +22,6 @@
  * preparation time is less, and so the total prepare + execute time is
  * competitive for queries over small data sets.
  */
-package net.hydromatic.optiq.impl.interpreter;
+package org.apache.calcite.adapter.interpreter;
 
 // End package-info.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/jdbc/CalciteConnection.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/jdbc/CalciteConnection.java b/core/src/main/java/org/apache/calcite/jdbc/CalciteConnection.java
index 402f72d..5301b3b 100644
--- a/core/src/main/java/org/apache/calcite/jdbc/CalciteConnection.java
+++ b/core/src/main/java/org/apache/calcite/jdbc/CalciteConnection.java
@@ -14,13 +14,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.jdbc;
+package org.apache.calcite.jdbc;
 
-import net.hydromatic.linq4j.QueryProvider;
-
-import net.hydromatic.optiq.SchemaPlus;
-import net.hydromatic.optiq.config.OptiqConnectionConfig;
-import net.hydromatic.optiq.impl.java.JavaTypeFactory;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.config.CalciteConnectionConfig;
+import org.apache.calcite.linq4j.QueryProvider;
+import org.apache.calcite.schema.SchemaPlus;
 
 import java.sql.Connection;
 import java.sql.SQLException;
@@ -34,11 +33,11 @@ import java.util.Properties;
  * <p>You can start off with an empty connection (no schemas), define one
  * or two schemas, and start querying them.</p>
  *
- * <p>Since an {@code OptiqConnection} implements the linq4j
+ * <p>Since a {@code CalciteConnection} implements the linq4j
  * {@link QueryProvider} interface, you can use a connection to execute
  * expression trees as queries.</p>
  */
-public interface OptiqConnection extends Connection, QueryProvider {
+public interface CalciteConnection extends Connection, QueryProvider {
   /**
    * Returns the root schema.
    *
@@ -76,7 +75,7 @@ public interface OptiqConnection extends Connection, QueryProvider {
   // in java.sql.Connection from JDK 1.7, but declare here to allow other JDKs
   String getSchema() throws SQLException;
 
-  OptiqConnectionConfig config();
+  CalciteConnectionConfig config();
 }
 
-// End OptiqConnection.java
+// End CalciteConnection.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 4e8ee07..769ffe8 100644
--- a/core/src/main/java/org/apache/calcite/jdbc/CalciteConnectionImpl.java
+++ b/core/src/main/java/org/apache/calcite/jdbc/CalciteConnectionImpl.java
@@ -14,41 +14,58 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.jdbc;
-
-import net.hydromatic.avatica.*;
-
-import net.hydromatic.linq4j.*;
-import net.hydromatic.linq4j.expressions.Expression;
-import net.hydromatic.linq4j.expressions.Expressions;
-import net.hydromatic.linq4j.function.Function0;
-
-import net.hydromatic.optiq.*;
-import net.hydromatic.optiq.config.OptiqConnectionConfig;
-import net.hydromatic.optiq.config.OptiqConnectionConfigImpl;
-import net.hydromatic.optiq.impl.AbstractSchema;
-import net.hydromatic.optiq.impl.java.JavaTypeFactory;
-import net.hydromatic.optiq.materialize.Lattice;
-import net.hydromatic.optiq.materialize.MaterializationService;
-import net.hydromatic.optiq.prepare.OptiqCatalogReader;
-import net.hydromatic.optiq.runtime.Hook;
-import net.hydromatic.optiq.server.OptiqServer;
-import net.hydromatic.optiq.server.OptiqServerStatement;
-
-import org.eigenbase.reltype.RelDataTypeSystem;
-import org.eigenbase.sql.advise.SqlAdvisor;
-import org.eigenbase.sql.advise.SqlAdvisorValidator;
-import org.eigenbase.sql.fun.SqlStdOperatorTable;
-import org.eigenbase.sql.validate.SqlConformance;
-import org.eigenbase.sql.validate.SqlValidatorWithHints;
-import org.eigenbase.util.Holder;
-
-import com.google.common.collect.*;
+package org.apache.calcite.jdbc;
+
+import org.apache.calcite.DataContext;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.avatica.AvaticaConnection;
+import org.apache.calcite.avatica.AvaticaFactory;
+import org.apache.calcite.avatica.AvaticaParameter;
+import org.apache.calcite.avatica.AvaticaPrepareResult;
+import org.apache.calcite.avatica.AvaticaStatement;
+import org.apache.calcite.avatica.Helper;
+import org.apache.calcite.avatica.InternalProperty;
+import org.apache.calcite.avatica.Meta;
+import org.apache.calcite.avatica.UnregisteredDriver;
+import org.apache.calcite.config.CalciteConnectionConfig;
+import org.apache.calcite.config.CalciteConnectionConfigImpl;
+import org.apache.calcite.linq4j.BaseQueryable;
+import org.apache.calcite.linq4j.Enumerator;
+import org.apache.calcite.linq4j.QueryProvider;
+import org.apache.calcite.linq4j.Queryable;
+import org.apache.calcite.linq4j.function.Function0;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.materialize.Lattice;
+import org.apache.calcite.materialize.MaterializationService;
+import org.apache.calcite.prepare.CalciteCatalogReader;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.runtime.Hook;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.schema.Schemas;
+import org.apache.calcite.schema.impl.AbstractSchema;
+import org.apache.calcite.server.CalciteServer;
+import org.apache.calcite.server.CalciteServerStatement;
+import org.apache.calcite.sql.advise.SqlAdvisor;
+import org.apache.calcite.sql.advise.SqlAdvisorValidator;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.validate.SqlConformance;
+import org.apache.calcite.sql.validate.SqlValidatorWithHints;
+import org.apache.calcite.util.BuiltInMethod;
+import org.apache.calcite.util.Holder;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
 
 import java.io.Serializable;
-import java.lang.reflect.*;
-import java.sql.*;
-import java.util.*;
+import java.lang.reflect.Type;
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.TimeZone;
 
 /**
  * Implementation of JDBC connection
@@ -56,20 +73,20 @@ import java.util.*;
  *
  * <p>Abstract to allow newer versions of JDBC to add methods.</p>
  */
-abstract class OptiqConnectionImpl
+abstract class CalciteConnectionImpl
     extends AvaticaConnection
-    implements OptiqConnection, QueryProvider {
+    implements CalciteConnection, QueryProvider {
   public final JavaTypeFactory typeFactory;
 
-  final OptiqRootSchema rootSchema;
-  final Function0<OptiqPrepare> prepareFactory;
-  final OptiqServer server = new OptiqServerImpl();
+  final CalciteRootSchema rootSchema;
+  final Function0<CalcitePrepare> prepareFactory;
+  final CalciteServer server = new CalciteServerImpl();
 
   // must be package-protected
   static final Trojan TROJAN = createTrojan();
 
   /**
-   * Creates an OptiqConnectionImpl.
+   * Creates a CalciteConnectionImpl.
    *
    * <p>Not public; method is called only from the driver.</p>
    *
@@ -80,11 +97,11 @@ abstract class OptiqConnectionImpl
    * @param rootSchema Root schema, or null
    * @param typeFactory Type factory, or null
    */
-  protected OptiqConnectionImpl(Driver driver, AvaticaFactory factory,
-      String url, Properties info, OptiqRootSchema rootSchema,
+  protected CalciteConnectionImpl(Driver driver, AvaticaFactory factory,
+      String url, Properties info, CalciteRootSchema rootSchema,
       JavaTypeFactory typeFactory) {
     super(driver, factory, url, info);
-    OptiqConnectionConfig cfg = new OptiqConnectionConfigImpl(info);
+    CalciteConnectionConfig cfg = new CalciteConnectionConfigImpl(info);
     this.prepareFactory = driver.prepareFactory;
     if (typeFactory != null) {
       this.typeFactory = typeFactory;
@@ -94,7 +111,7 @@ abstract class OptiqConnectionImpl
       this.typeFactory = new JavaTypeFactoryImpl(typeSystem);
     }
     this.rootSchema =
-        rootSchema != null ? rootSchema : OptiqSchema.createRootSchema(true);
+        rootSchema != null ? rootSchema : CalciteSchema.createRootSchema(true);
 
     this.properties.put(InternalProperty.CASE_SENSITIVE, cfg.caseSensitive());
     this.properties.put(InternalProperty.UNQUOTED_CASING, cfg.unquotedCasing());
@@ -110,15 +127,15 @@ abstract class OptiqConnectionImpl
     return (MetaImpl) meta;
   }
 
-  public OptiqConnectionConfig config() {
-    return new OptiqConnectionConfigImpl(info);
+  public CalciteConnectionConfig config() {
+    return new CalciteConnectionConfigImpl(info);
   }
 
   /** Called after the constructor has completed and the model has been
    * loaded. */
   void init() {
     final MaterializationService service = MaterializationService.instance();
-    for (OptiqSchema.LatticeEntry e : Schemas.getLatticeEntries(rootSchema)) {
+    for (CalciteSchema.LatticeEntry e : Schemas.getLatticeEntries(rootSchema)) {
       final Lattice lattice = e.getLattice();
       for (Lattice.Tile tile : lattice.computeTiles()) {
         service.defineTile(lattice, tile.bitSet(), tile.measures, e.schema,
@@ -129,8 +146,8 @@ abstract class OptiqConnectionImpl
 
   @Override public AvaticaStatement createStatement(int resultSetType,
       int resultSetConcurrency, int resultSetHoldability) throws SQLException {
-    OptiqStatement statement =
-        (OptiqStatement) super.createStatement(
+    CalciteStatement statement =
+        (CalciteStatement) super.createStatement(
             resultSetType, resultSetConcurrency, resultSetHoldability);
     server.addStatement(statement);
     return statement;
@@ -144,8 +161,8 @@ abstract class OptiqConnectionImpl
     try {
       AvaticaPrepareResult prepareResult =
           parseQuery(sql, new ContextImpl(this), -1);
-      OptiqPreparedStatement statement =
-          (OptiqPreparedStatement) factory.newPreparedStatement(
+      CalcitePreparedStatement statement =
+          (CalcitePreparedStatement) factory.newPreparedStatement(
               this,
               prepareResult,
               resultSetType,
@@ -162,19 +179,19 @@ abstract class OptiqConnectionImpl
     }
   }
 
-  <T> OptiqPrepare.PrepareResult<T> parseQuery(String sql,
-      OptiqPrepare.Context prepareContext, int maxRowCount) {
-    OptiqPrepare.Dummy.push(prepareContext);
+  <T> CalcitePrepare.PrepareResult<T> parseQuery(String sql,
+      CalcitePrepare.Context prepareContext, int maxRowCount) {
+    CalcitePrepare.Dummy.push(prepareContext);
     try {
-      final OptiqPrepare prepare = prepareFactory.apply();
+      final CalcitePrepare prepare = prepareFactory.apply();
       return prepare.prepareSql(prepareContext, sql, null, Object[].class,
           maxRowCount);
     } finally {
-      OptiqPrepare.Dummy.pop(prepareContext);
+      CalcitePrepare.Dummy.pop(prepareContext);
     }
   }
 
-  // OptiqConnection methods
+  // CalciteConnection methods
 
   public SchemaPlus getRootSchema() {
     return rootSchema.plus();
@@ -192,11 +209,11 @@ abstract class OptiqConnectionImpl
 
   public <T> Queryable<T> createQuery(
       Expression expression, Class<T> rowType) {
-    return new OptiqQueryable<T>(this, rowType, expression);
+    return new CalciteQueryable<T>(this, rowType, expression);
   }
 
   public <T> Queryable<T> createQuery(Expression expression, Type rowType) {
-    return new OptiqQueryable<T>(this, rowType, expression);
+    return new CalciteQueryable<T>(this, rowType, expression);
   }
 
   public <T> T execute(Expression expression, Type type) {
@@ -209,8 +226,8 @@ abstract class OptiqConnectionImpl
 
   public <T> Enumerator<T> executeQuery(Queryable<T> queryable) {
     try {
-      OptiqStatement statement = (OptiqStatement) createStatement();
-      OptiqPrepare.PrepareResult<T> enumerable =
+      CalciteStatement statement = (CalciteStatement) createStatement();
+      CalcitePrepare.PrepareResult<T> enumerable =
           statement.prepare(queryable);
       final DataContext dataContext =
           createDataContext(ImmutableMap.<String, Object>of());
@@ -238,28 +255,27 @@ abstract class OptiqConnectionImpl
   }
 
   /** Implementation of Queryable. */
-  static class OptiqQueryable<T>
-      extends BaseQueryable<T> {
-    public OptiqQueryable(
-        OptiqConnection connection, Type elementType, Expression expression) {
+  static class CalciteQueryable<T> extends BaseQueryable<T> {
+    public CalciteQueryable(CalciteConnection connection, Type elementType,
+        Expression expression) {
       super(connection, elementType, expression);
     }
 
-    public OptiqConnection getConnection() {
-      return (OptiqConnection) provider;
+    public CalciteConnection getConnection() {
+      return (CalciteConnection) provider;
     }
   }
 
   /** Implementation of Server. */
-  private static class OptiqServerImpl implements OptiqServer {
-    final List<OptiqServerStatement> statementList =
-        new ArrayList<OptiqServerStatement>();
+  private static class CalciteServerImpl implements CalciteServer {
+    final List<CalciteServerStatement> statementList =
+        new ArrayList<CalciteServerStatement>();
 
-    public void removeStatement(OptiqServerStatement optiqServerStatement) {
-      statementList.add(optiqServerStatement);
+    public void removeStatement(CalciteServerStatement calciteServerStatement) {
+      statementList.add(calciteServerStatement);
     }
 
-    public void addStatement(OptiqServerStatement statement) {
+    public void addStatement(CalciteServerStatement statement) {
       statementList.add(statement);
     }
   }
@@ -274,18 +290,18 @@ abstract class OptiqConnectionImpl
         String name) {
       return Expressions.call(
           DataContext.ROOT,
-          BuiltinMethod.DATA_CONTEXT_GET_ROOT_SCHEMA.method);
+          BuiltInMethod.DATA_CONTEXT_GET_ROOT_SCHEMA.method);
     }
   }
 
   /** Implementation of DataContext. */
   static class DataContextImpl implements DataContext {
     private final ImmutableMap<Object, Object> map;
-    private final OptiqSchema rootSchema;
+    private final CalciteSchema rootSchema;
     private final QueryProvider queryProvider;
     private final JavaTypeFactory typeFactory;
 
-    DataContextImpl(OptiqConnectionImpl connection,
+    DataContextImpl(CalciteConnectionImpl connection,
         Map<String, Object> parameters) {
       this.queryProvider = connection;
       this.typeFactory = connection.getTypeFactory();
@@ -330,7 +346,7 @@ abstract class OptiqConnectionImpl
     }
 
     private SqlAdvisor getSqlAdvisor() {
-      final OptiqConnectionImpl con = (OptiqConnectionImpl) queryProvider;
+      final CalciteConnectionImpl con = (CalciteConnectionImpl) queryProvider;
       final String schemaName = con.getSchema();
       final List<String> schemaPath =
           schemaName == null
@@ -338,7 +354,7 @@ abstract class OptiqConnectionImpl
               : ImmutableList.of(schemaName);
       final SqlValidatorWithHints validator =
           new SqlAdvisorValidator(SqlStdOperatorTable.instance(),
-          new OptiqCatalogReader(rootSchema, con.config().caseSensitive(),
+          new CalciteCatalogReader(rootSchema, con.config().caseSensitive(),
               schemaPath, typeFactory),
           typeFactory, SqlConformance.DEFAULT);
       return new SqlAdvisor(validator);
@@ -358,10 +374,10 @@ abstract class OptiqConnectionImpl
   }
 
   /** Implementation of Context. */
-  static class ContextImpl implements OptiqPrepare.Context {
-    private final OptiqConnectionImpl connection;
+  static class ContextImpl implements CalcitePrepare.Context {
+    private final CalciteConnectionImpl connection;
 
-    public ContextImpl(OptiqConnectionImpl connection) {
+    public ContextImpl(CalciteConnectionImpl connection) {
       this.connection = connection;
     }
 
@@ -369,7 +385,7 @@ abstract class OptiqConnectionImpl
       return connection.typeFactory;
     }
 
-    public OptiqRootSchema getRootSchema() {
+    public CalciteRootSchema getRootSchema() {
       return connection.rootSchema;
     }
 
@@ -380,7 +396,7 @@ abstract class OptiqConnectionImpl
           : ImmutableList.of(schemaName);
     }
 
-    public OptiqConnectionConfig config() {
+    public CalciteConnectionConfig config() {
       return connection.config();
     }
 
@@ -388,9 +404,9 @@ abstract class OptiqConnectionImpl
       return connection.createDataContext(ImmutableMap.<String, Object>of());
     }
 
-    public OptiqPrepare.SparkHandler spark() {
+    public CalcitePrepare.SparkHandler spark() {
       final boolean enable = config().spark();
-      return OptiqPrepare.Dummy.getSparkHandler(enable);
+      return CalcitePrepare.Dummy.getSparkHandler(enable);
     }
   }
 
@@ -416,4 +432,4 @@ abstract class OptiqConnectionImpl
 
 }
 
-// End OptiqConnectionImpl.java
+// End CalciteConnectionImpl.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/jdbc/CalciteFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/jdbc/CalciteFactory.java b/core/src/main/java/org/apache/calcite/jdbc/CalciteFactory.java
index cac1678..03f5496 100644
--- a/core/src/main/java/org/apache/calcite/jdbc/CalciteFactory.java
+++ b/core/src/main/java/org/apache/calcite/jdbc/CalciteFactory.java
@@ -14,24 +14,25 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.jdbc;
+package org.apache.calcite.jdbc;
 
-import net.hydromatic.avatica.*;
-
-import net.hydromatic.optiq.impl.java.JavaTypeFactory;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.avatica.AvaticaConnection;
+import org.apache.calcite.avatica.AvaticaFactory;
+import org.apache.calcite.avatica.UnregisteredDriver;
 
 import java.util.Properties;
 
 /**
- * Extension of {@link net.hydromatic.avatica.AvaticaFactory}
+ * Extension of {@link org.apache.calcite.avatica.AvaticaFactory}
  * for Calcite.
  */
-public abstract class OptiqFactory implements AvaticaFactory {
+public abstract class CalciteFactory implements AvaticaFactory {
   protected final int major;
   protected final int minor;
 
   /** Creates a JDBC factory with given major/minor version number. */
-  protected OptiqFactory(int major, int minor) {
+  protected CalciteFactory(int major, int minor) {
     this.major = major;
     this.minor = minor;
   }
@@ -55,7 +56,7 @@ public abstract class OptiqFactory implements AvaticaFactory {
   /** Creates a connection with a root schema. */
   public abstract AvaticaConnection newConnection(UnregisteredDriver driver,
       AvaticaFactory factory, String url, Properties info,
-      OptiqRootSchema rootSchema, JavaTypeFactory typeFactory);
+      CalciteRootSchema rootSchema, JavaTypeFactory typeFactory);
 }
 
-// End OptiqFactory.java
+// End CalciteFactory.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/jdbc/CalciteJdbc40Factory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/jdbc/CalciteJdbc40Factory.java b/core/src/main/java/org/apache/calcite/jdbc/CalciteJdbc40Factory.java
index a861b07..3d1404d 100644
--- a/core/src/main/java/org/apache/calcite/jdbc/CalciteJdbc40Factory.java
+++ b/core/src/main/java/org/apache/calcite/jdbc/CalciteJdbc40Factory.java
@@ -14,17 +14,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.jdbc;
+package org.apache.calcite.jdbc;
 
 /**
- * Implementation of {@link net.hydromatic.avatica.AvaticaFactory}
+ * Implementation of {@link org.apache.calcite.avatica.AvaticaFactory}
  * for Calcite and JDBC 4.0 (corresponds to JDK 1.6).
  */
-public class OptiqJdbc40Factory extends OptiqJdbc41Factory {
+public class CalciteJdbc40Factory extends CalciteJdbc41Factory {
   /** Creates a factory for JDBC version 4.1. */
-  public OptiqJdbc40Factory() {
+  public CalciteJdbc40Factory() {
     super(4, 0);
   }
 }
 
-// End OptiqJdbc40Factory.java
+// End CalciteJdbc40Factory.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/jdbc/CalciteJdbc41Factory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/jdbc/CalciteJdbc41Factory.java b/core/src/main/java/org/apache/calcite/jdbc/CalciteJdbc41Factory.java
index b55708f..73efcaa 100644
--- a/core/src/main/java/org/apache/calcite/jdbc/CalciteJdbc41Factory.java
+++ b/core/src/main/java/org/apache/calcite/jdbc/CalciteJdbc41Factory.java
@@ -14,55 +14,66 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.jdbc;
-
-import net.hydromatic.avatica.*;
-
-import net.hydromatic.optiq.impl.java.JavaTypeFactory;
+package org.apache.calcite.jdbc;
+
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.avatica.AvaticaConnection;
+import org.apache.calcite.avatica.AvaticaDatabaseMetaData;
+import org.apache.calcite.avatica.AvaticaFactory;
+import org.apache.calcite.avatica.AvaticaPrepareResult;
+import org.apache.calcite.avatica.AvaticaPreparedStatement;
+import org.apache.calcite.avatica.AvaticaResultSetMetaData;
+import org.apache.calcite.avatica.AvaticaStatement;
+import org.apache.calcite.avatica.ColumnMetaData;
+import org.apache.calcite.avatica.UnregisteredDriver;
 
 import java.io.InputStream;
 import java.io.Reader;
-import java.sql.*;
+import java.sql.NClob;
+import java.sql.ResultSetMetaData;
+import java.sql.RowId;
+import java.sql.SQLException;
+import java.sql.SQLXML;
 import java.util.List;
 import java.util.Properties;
 import java.util.TimeZone;
 
 /**
- * Implementation of {@link net.hydromatic.avatica.AvaticaFactory}
+ * Implementation of {@link org.apache.calcite.avatica.AvaticaFactory}
  * for Calcite and JDBC 4.1 (corresponds to JDK 1.7).
  */
 @SuppressWarnings("UnusedDeclaration")
-public class OptiqJdbc41Factory extends OptiqFactory {
+public class CalciteJdbc41Factory extends CalciteFactory {
   /** Creates a factory for JDBC version 4.1. */
-  public OptiqJdbc41Factory() {
+  public CalciteJdbc41Factory() {
     this(4, 1);
   }
 
   /** Creates a JDBC factory with given major/minor version number. */
-  protected OptiqJdbc41Factory(int major, int minor) {
+  protected CalciteJdbc41Factory(int major, int minor) {
     super(major, minor);
   }
 
-  public OptiqJdbc41Connection newConnection(UnregisteredDriver driver,
+  public CalciteJdbc41Connection newConnection(UnregisteredDriver driver,
       AvaticaFactory factory, String url, Properties info,
-      OptiqRootSchema rootSchema, JavaTypeFactory typeFactory) {
-    return new OptiqJdbc41Connection(
+      CalciteRootSchema rootSchema, JavaTypeFactory typeFactory) {
+    return new CalciteJdbc41Connection(
         (Driver) driver, factory, url, info, rootSchema, typeFactory);
   }
 
-  public OptiqJdbc41DatabaseMetaData newDatabaseMetaData(
+  public CalciteJdbc41DatabaseMetaData newDatabaseMetaData(
       AvaticaConnection connection) {
-    return new OptiqJdbc41DatabaseMetaData(
-        (OptiqConnectionImpl) connection);
+    return new CalciteJdbc41DatabaseMetaData(
+        (CalciteConnectionImpl) connection);
   }
 
-  public OptiqJdbc41Statement newStatement(
+  public CalciteJdbc41Statement newStatement(
       AvaticaConnection connection,
       int resultSetType,
       int resultSetConcurrency,
       int resultSetHoldability) {
-    return new OptiqJdbc41Statement(
-        (OptiqConnectionImpl) connection, resultSetType, resultSetConcurrency,
+    return new CalciteJdbc41Statement(
+        (CalciteConnectionImpl) connection, resultSetType, resultSetConcurrency,
         resultSetHoldability);
   }
 
@@ -72,20 +83,20 @@ public class OptiqJdbc41Factory extends OptiqFactory {
       int resultSetType,
       int resultSetConcurrency,
       int resultSetHoldability) throws SQLException {
-    return new OptiqJdbc41PreparedStatement(
-        (OptiqConnectionImpl) connection,
-        (OptiqPrepare.PrepareResult) prepareResult, resultSetType,
+    return new CalciteJdbc41PreparedStatement(
+        (CalciteConnectionImpl) connection,
+        (CalcitePrepare.PrepareResult) prepareResult, resultSetType,
         resultSetConcurrency, resultSetHoldability);
   }
 
-  public OptiqResultSet newResultSet(
+  public CalciteResultSet newResultSet(
       AvaticaStatement statement,
       AvaticaPrepareResult prepareResult,
       TimeZone timeZone) {
     final ResultSetMetaData metaData =
         newResultSetMetaData(statement, prepareResult.getColumnList());
-    return new OptiqResultSet(statement,
-        (OptiqPrepare.PrepareResult) prepareResult, metaData, timeZone);
+    return new CalciteResultSet(statement,
+        (CalcitePrepare.PrepareResult) prepareResult, metaData, timeZone);
   }
 
   public ResultSetMetaData newResultSetMetaData(
@@ -96,18 +107,18 @@ public class OptiqJdbc41Factory extends OptiqFactory {
   }
 
   /** Implementation of connection for JDBC 4.1. */
-  private static class OptiqJdbc41Connection extends OptiqConnectionImpl {
-    OptiqJdbc41Connection(Driver driver, AvaticaFactory factory, String url,
-        Properties info, OptiqRootSchema rootSchema,
+  private static class CalciteJdbc41Connection extends CalciteConnectionImpl {
+    CalciteJdbc41Connection(Driver driver, AvaticaFactory factory, String url,
+        Properties info, CalciteRootSchema rootSchema,
         JavaTypeFactory typeFactory) {
       super(driver, factory, url, info, rootSchema, typeFactory);
     }
   }
 
   /** Implementation of statement for JDBC 4.1. */
-  private static class OptiqJdbc41Statement extends OptiqStatement {
-    public OptiqJdbc41Statement(
-        OptiqConnectionImpl connection,
+  private static class CalciteJdbc41Statement extends CalciteStatement {
+    public CalciteJdbc41Statement(
+        CalciteConnectionImpl connection,
         int resultSetType,
         int resultSetConcurrency,
         int resultSetHoldability) {
@@ -117,11 +128,11 @@ public class OptiqJdbc41Factory extends OptiqFactory {
   }
 
   /** Implementation of prepared statement for JDBC 4.1. */
-  private static class OptiqJdbc41PreparedStatement
-      extends OptiqPreparedStatement {
-    OptiqJdbc41PreparedStatement(
-        OptiqConnectionImpl connection,
-        OptiqPrepare.PrepareResult prepareResult,
+  private static class CalciteJdbc41PreparedStatement
+      extends CalcitePreparedStatement {
+    CalciteJdbc41PreparedStatement(
+        CalciteConnectionImpl connection,
+        CalcitePrepare.PrepareResult prepareResult,
         int resultSetType,
         int resultSetConcurrency,
         int resultSetHoldability) throws SQLException {
@@ -239,12 +250,12 @@ public class OptiqJdbc41Factory extends OptiqFactory {
   }
 
   /** Implementation of database metadata for JDBC 4.1. */
-  private static class OptiqJdbc41DatabaseMetaData
+  private static class CalciteJdbc41DatabaseMetaData
       extends AvaticaDatabaseMetaData {
-    OptiqJdbc41DatabaseMetaData(OptiqConnectionImpl connection) {
+    CalciteJdbc41DatabaseMetaData(CalciteConnectionImpl connection) {
       super(connection);
     }
   }
 }
 
-// End OptiqJdbc41Factory.java
+// End CalciteJdbc41Factory.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/jdbc/CalcitePrepare.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/jdbc/CalcitePrepare.java b/core/src/main/java/org/apache/calcite/jdbc/CalcitePrepare.java
index 2814546..9f17655 100644
--- a/core/src/main/java/org/apache/calcite/jdbc/CalcitePrepare.java
+++ b/core/src/main/java/org/apache/calcite/jdbc/CalcitePrepare.java
@@ -14,48 +14,56 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.jdbc;
-
-import net.hydromatic.avatica.*;
-
-import net.hydromatic.linq4j.*;
-import net.hydromatic.linq4j.expressions.ClassDeclaration;
-import net.hydromatic.linq4j.function.Function0;
-
-import net.hydromatic.optiq.*;
-import net.hydromatic.optiq.config.OptiqConnectionConfig;
-import net.hydromatic.optiq.impl.java.JavaTypeFactory;
-import net.hydromatic.optiq.prepare.OptiqPrepareImpl;
-import net.hydromatic.optiq.runtime.*;
-
-import org.eigenbase.rel.RelNode;
-import org.eigenbase.relopt.RelOptPlanner;
-import org.eigenbase.relopt.RelOptRule;
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.reltype.RelDataTypeFactory;
-import org.eigenbase.sql.SqlNode;
-import org.eigenbase.sql.validate.SqlValidator;
-import org.eigenbase.util.Stacks;
+package org.apache.calcite.jdbc;
+
+import org.apache.calcite.DataContext;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.avatica.AvaticaParameter;
+import org.apache.calcite.avatica.AvaticaPrepareResult;
+import org.apache.calcite.avatica.ColumnMetaData;
+import org.apache.calcite.avatica.Cursor;
+import org.apache.calcite.config.CalciteConnectionConfig;
+import org.apache.calcite.linq4j.Enumerable;
+import org.apache.calcite.linq4j.EnumerableDefaults;
+import org.apache.calcite.linq4j.Enumerator;
+import org.apache.calcite.linq4j.Queryable;
+import org.apache.calcite.linq4j.function.Function0;
+import org.apache.calcite.linq4j.tree.ClassDeclaration;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.prepare.CalcitePrepareImpl;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.runtime.ArrayEnumeratorCursor;
+import org.apache.calcite.runtime.Bindable;
+import org.apache.calcite.runtime.ObjectEnumeratorCursor;
+import org.apache.calcite.runtime.RecordEnumeratorCursor;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.util.Stacks;
 
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
 import java.lang.reflect.Type;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
 
 /**
  * API for a service that prepares statements for execution.
  */
-public interface OptiqPrepare {
-  Function0<OptiqPrepare> DEFAULT_FACTORY =
-      new Function0<OptiqPrepare>() {
-        public OptiqPrepare apply() {
-          return new OptiqPrepareImpl();
+public interface CalcitePrepare {
+  Function0<CalcitePrepare> DEFAULT_FACTORY =
+      new Function0<CalcitePrepare>() {
+        public CalcitePrepare apply() {
+          return new CalcitePrepareImpl();
         }
       };
   ThreadLocal<ArrayList<Context>> THREAD_CONTEXT_STACK =
       new ThreadLocal<ArrayList<Context>>() {
-        @Override
-        protected ArrayList<Context> initialValue() {
+        @Override protected ArrayList<Context> initialValue() {
           return new ArrayList<Context>();
         }
       };
@@ -79,11 +87,11 @@ public interface OptiqPrepare {
   interface Context {
     JavaTypeFactory getTypeFactory();
 
-    OptiqRootSchema getRootSchema();
+    CalciteRootSchema getRootSchema();
 
     List<String> getDefaultSchemaPath();
 
-    OptiqConnectionConfig config();
+    CalciteConnectionConfig config();
 
     /** Returns the spark handler. Never null. */
     SparkHandler spark();
@@ -130,9 +138,9 @@ public interface OptiqPrepare {
     private static SparkHandler createHandler() {
       try {
         final Class<?> clazz =
-            Class.forName("net.hydromatic.optiq.impl.spark.SparkHandlerImpl");
+            Class.forName("org.apache.calcite.adapter.spark.SparkHandlerImpl");
         Method method = clazz.getMethod("instance");
-        return (OptiqPrepare.SparkHandler) method.invoke(null);
+        return (CalcitePrepare.SparkHandler) method.invoke(null);
       } catch (ClassNotFoundException e) {
         return new TrivialSparkHandler();
       } catch (IllegalAccessException e) {
@@ -185,13 +193,13 @@ public interface OptiqPrepare {
 
   /** The result of parsing and validating a SQL query. */
   public static class ParseResult {
-    public final OptiqPrepareImpl prepare;
+    public final CalcitePrepareImpl prepare;
     public final String sql; // for debug
     public final SqlNode sqlNode;
     public final RelDataType rowType;
     public final RelDataTypeFactory typeFactory;
 
-    public ParseResult(OptiqPrepareImpl prepare, SqlValidator validator,
+    public ParseResult(CalcitePrepareImpl prepare, SqlValidator validator,
         String sql,
         SqlNode sqlNode, RelDataType rowType) {
       super();
@@ -208,7 +216,7 @@ public interface OptiqPrepare {
   public static class ConvertResult extends ParseResult {
     public final RelNode relNode;
 
-    public ConvertResult(OptiqPrepareImpl prepare, SqlValidator validator,
+    public ConvertResult(CalcitePrepareImpl prepare, SqlValidator validator,
         String sql, SqlNode sqlNode, RelDataType rowType, RelNode relNode) {
       super(prepare, validator, sql, sqlNode, rowType);
       this.relNode = relNode;
@@ -293,4 +301,4 @@ public interface OptiqPrepare {
   }
 }
 
-// End OptiqPrepare.java
+// End CalcitePrepare.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/jdbc/CalcitePreparedStatement.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/jdbc/CalcitePreparedStatement.java b/core/src/main/java/org/apache/calcite/jdbc/CalcitePreparedStatement.java
index 3c66e92..86e657b 100644
--- a/core/src/main/java/org/apache/calcite/jdbc/CalcitePreparedStatement.java
+++ b/core/src/main/java/org/apache/calcite/jdbc/CalcitePreparedStatement.java
@@ -14,35 +14,35 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.jdbc;
+package org.apache.calcite.jdbc;
 
-import net.hydromatic.avatica.AvaticaPrepareResult;
-import net.hydromatic.avatica.AvaticaPreparedStatement;
+import org.apache.calcite.avatica.AvaticaPrepareResult;
+import org.apache.calcite.avatica.AvaticaPreparedStatement;
+import org.apache.calcite.server.CalciteServerStatement;
 
-import net.hydromatic.optiq.server.OptiqServerStatement;
-
-import java.sql.*;
+import java.sql.SQLException;
 
 /**
  * Implementation of {@link java.sql.PreparedStatement}
  * for the Calcite engine.
  *
  * <p>This class has sub-classes which implement JDBC 3.0 and JDBC 4.0 APIs;
- * it is instantiated using {@link net.hydromatic.avatica.AvaticaFactory#newPreparedStatement}.</p>
+ * it is instantiated using
+ * {@link org.apache.calcite.avatica.AvaticaFactory#newPreparedStatement}.
  */
-abstract class OptiqPreparedStatement
+abstract class CalcitePreparedStatement
     extends AvaticaPreparedStatement
-    implements OptiqServerStatement {
+    implements CalciteServerStatement {
   /**
-   * Creates an OptiqPreparedStatement.
+   * Creates a CalcitePreparedStatement.
    *
    * @param connection Connection
    * @param prepareResult Result of preparing statement
    *
    * @throws SQLException if database error occurs
    */
-  protected OptiqPreparedStatement(
-      OptiqConnectionImpl connection,
+  protected CalcitePreparedStatement(
+      CalciteConnectionImpl connection,
       AvaticaPrepareResult prepareResult,
       int resultSetType,
       int resultSetConcurrency,
@@ -53,14 +53,13 @@ abstract class OptiqPreparedStatement
         resultSetHoldability);
   }
 
-  @Override
-  public OptiqConnectionImpl getConnection() {
-    return (OptiqConnectionImpl) super.getConnection();
+  @Override public CalciteConnectionImpl getConnection() {
+    return (CalciteConnectionImpl) super.getConnection();
   }
 
-  public OptiqConnectionImpl.ContextImpl createPrepareContext() {
-    return new OptiqConnectionImpl.ContextImpl(getConnection());
+  public CalciteConnectionImpl.ContextImpl createPrepareContext() {
+    return new CalciteConnectionImpl.ContextImpl(getConnection());
   }
 }
 
-// End OptiqPreparedStatement.java
+// End CalcitePreparedStatement.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/jdbc/CalciteResultSet.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/jdbc/CalciteResultSet.java b/core/src/main/java/org/apache/calcite/jdbc/CalciteResultSet.java
index 347542f..3e16c39 100644
--- a/core/src/main/java/org/apache/calcite/jdbc/CalciteResultSet.java
+++ b/core/src/main/java/org/apache/calcite/jdbc/CalciteResultSet.java
@@ -14,36 +14,40 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.jdbc;
+package org.apache.calcite.jdbc;
 
-import net.hydromatic.avatica.*;
-
-import net.hydromatic.linq4j.Enumerator;
-import net.hydromatic.linq4j.Linq4j;
-
-import net.hydromatic.optiq.runtime.*;
+import org.apache.calcite.avatica.AvaticaResultSet;
+import org.apache.calcite.avatica.AvaticaStatement;
+import org.apache.calcite.avatica.ColumnMetaData;
+import org.apache.calcite.avatica.Cursor;
+import org.apache.calcite.avatica.Handler;
+import org.apache.calcite.linq4j.Enumerator;
+import org.apache.calcite.linq4j.Linq4j;
+import org.apache.calcite.runtime.ArrayEnumeratorCursor;
+import org.apache.calcite.runtime.ObjectEnumeratorCursor;
 
 import com.google.common.collect.ImmutableList;
 
-import java.sql.*;
-import java.util.*;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.TimeZone;
 
 /**
  * Implementation of {@link ResultSet}
  * for the Calcite engine.
  */
-public class OptiqResultSet extends AvaticaResultSet {
-  OptiqResultSet(
-      AvaticaStatement statement,
-      OptiqPrepare.PrepareResult prepareResult,
-      ResultSetMetaData resultSetMetaData,
-      TimeZone timeZone) {
+public class CalciteResultSet extends AvaticaResultSet {
+  CalciteResultSet(AvaticaStatement statement,
+      CalcitePrepare.PrepareResult prepareResult,
+      ResultSetMetaData resultSetMetaData, TimeZone timeZone) {
     super(statement, prepareResult, resultSetMetaData, timeZone);
   }
 
-  @Override protected OptiqResultSet execute() throws SQLException {
+  @Override protected CalciteResultSet execute() throws SQLException {
     // Call driver's callback. It is permitted to throw a RuntimeException.
-    OptiqConnectionImpl connection = getOptiqConnection();
+    CalciteConnectionImpl connection = getCalciteConnection();
     final boolean autoTemp = connection.config().autoTemp();
     Handler.ResultSink resultSink = null;
     if (autoTemp) {
@@ -52,8 +56,7 @@ public class OptiqResultSet extends AvaticaResultSet {
         }
       };
     }
-    connection.getDriver().handler.onStatementExecute(
-        statement, resultSink);
+    connection.getDriver().handler.onStatementExecute(statement, resultSink);
 
     super.execute();
     return this;
@@ -61,9 +64,9 @@ public class OptiqResultSet extends AvaticaResultSet {
 
   @Override public ResultSet create(ColumnMetaData.AvaticaType elementType,
       Iterable iterable) {
-    final OptiqResultSet resultSet =
-        new OptiqResultSet(statement,
-            (OptiqPrepare.PrepareResult) prepareResult, resultSetMetaData,
+    final CalciteResultSet resultSet =
+        new CalciteResultSet(statement,
+            (CalcitePrepare.PrepareResult) prepareResult, resultSetMetaData,
             localCalendar.getTimeZone());
     final Cursor cursor = resultSet.createCursor(elementType, iterable);
     final List<ColumnMetaData> columnMetaDataList;
@@ -87,14 +90,14 @@ public class OptiqResultSet extends AvaticaResultSet {
   }
 
   // do not make public
-  OptiqPrepare.PrepareResult getPrepareResult() {
-    return (OptiqPrepare.PrepareResult) prepareResult;
+  CalcitePrepare.PrepareResult getPrepareResult() {
+    return (CalcitePrepare.PrepareResult) prepareResult;
   }
 
   // do not make public
-  OptiqConnectionImpl getOptiqConnection() {
-    return (OptiqConnectionImpl) statement.getConnection();
+  CalciteConnectionImpl getCalciteConnection() {
+    return (CalciteConnectionImpl) statement.getConnection();
   }
 }
 
-// End OptiqResultSet.java
+// End CalciteResultSet.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/jdbc/CalciteRootSchema.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/jdbc/CalciteRootSchema.java b/core/src/main/java/org/apache/calcite/jdbc/CalciteRootSchema.java
index 84a6c77..6a22618 100644
--- a/core/src/main/java/org/apache/calcite/jdbc/CalciteRootSchema.java
+++ b/core/src/main/java/org/apache/calcite/jdbc/CalciteRootSchema.java
@@ -14,18 +14,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.jdbc;
+package org.apache.calcite.jdbc;
 
-import net.hydromatic.optiq.Schema;
+import org.apache.calcite.schema.Schema;
 
 /**
  * Root schema.
  */
-public class OptiqRootSchema extends OptiqSchema {
+public class CalciteRootSchema extends CalciteSchema {
   /** Creates a root schema. */
-  OptiqRootSchema(Schema schema) {
+  CalciteRootSchema(Schema schema) {
     super(null, schema, "");
   }
 }
 
-// End OptiqRootSchema.java
+// End CalciteRootSchema.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/jdbc/CalciteSchema.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/jdbc/CalciteSchema.java b/core/src/main/java/org/apache/calcite/jdbc/CalciteSchema.java
index 83044ce..4b34c3e 100644
--- a/core/src/main/java/org/apache/calcite/jdbc/CalciteSchema.java
+++ b/core/src/main/java/org/apache/calcite/jdbc/CalciteSchema.java
@@ -14,32 +14,49 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.jdbc;
-
-import net.hydromatic.linq4j.Linq4j;
-import net.hydromatic.linq4j.expressions.Expression;
-
-import net.hydromatic.optiq.*;
-import net.hydromatic.optiq.Table;
-import net.hydromatic.optiq.impl.MaterializedViewTable;
-import net.hydromatic.optiq.impl.StarTable;
-import net.hydromatic.optiq.materialize.Lattice;
-import net.hydromatic.optiq.util.Compatible;
-
-import org.eigenbase.util.Pair;
+package org.apache.calcite.jdbc;
+
+import org.apache.calcite.linq4j.Linq4j;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.materialize.Lattice;
+import org.apache.calcite.schema.Function;
+import org.apache.calcite.schema.Schema;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.schema.Table;
+import org.apache.calcite.schema.TableMacro;
+import org.apache.calcite.schema.impl.MaterializedViewTable;
+import org.apache.calcite.schema.impl.StarTable;
+import org.apache.calcite.util.Compatible;
+import org.apache.calcite.util.Pair;
 
 import com.google.common.base.Preconditions;
-import com.google.common.cache.*;
-import com.google.common.collect.*;
-
-import java.util.*;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSortedMap;
+import com.google.common.collect.ImmutableSortedSet;
+import com.google.common.collect.LinkedListMultimap;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Multimap;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.NavigableSet;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.TreeSet;
 
 /**
  * Schema.
  *
  * <p>Wrapper around user-defined schema used internally.</p>
  */
-public class OptiqSchema {
+public class CalciteSchema {
   /** Comparator that compares all strings differently, but if two strings are
    * equal in case-insensitive match they are right next to each other. In a
    * collection sorted on this comparator, we can find case-insensitive matches
@@ -56,7 +73,7 @@ public class OptiqSchema {
         }
       };
 
-  private final OptiqSchema parent;
+  private final CalciteSchema parent;
   public final Schema schema;
   public final String name;
   /** Tables explicitly defined in this schema. Does not include tables in
@@ -71,23 +88,23 @@ public class OptiqSchema {
       new TreeSet<String>(COMPARATOR);
   private final NavigableMap<String, FunctionEntry> nullaryFunctionMap =
       new TreeMap<String, FunctionEntry>(COMPARATOR);
-  private final NavigableMap<String, OptiqSchema> subSchemaMap =
-      new TreeMap<String, OptiqSchema>(COMPARATOR);
+  private final NavigableMap<String, CalciteSchema> subSchemaMap =
+      new TreeMap<String, CalciteSchema>(COMPARATOR);
   private ImmutableList<ImmutableList<String>> path;
   private boolean cache = true;
   private final Cached<SubSchemaCache> implicitSubSchemaCache;
   private final Cached<NavigableSet<String>> implicitTableCache;
   private final Cached<NavigableSet<String>> implicitFunctionCache;
 
-  public OptiqSchema(OptiqSchema parent, final Schema schema, String name) {
+  public CalciteSchema(CalciteSchema parent, final Schema schema, String name) {
     this.parent = parent;
     this.schema = schema;
     this.name = name;
-    assert (parent == null) == (this instanceof OptiqRootSchema);
+    assert (parent == null) == (this instanceof CalciteRootSchema);
     this.implicitSubSchemaCache =
         new AbstractCached<SubSchemaCache>() {
           public SubSchemaCache build() {
-            return new SubSchemaCache(OptiqSchema.this,
+            return new SubSchemaCache(CalciteSchema.this,
                 Compatible.INSTANCE.navigableSet(
                     ImmutableSortedSet.copyOf(COMPARATOR,
                         schema.getSubSchemaNames())));
@@ -114,9 +131,9 @@ public class OptiqSchema {
   /** Creates a root schema. When <code>addMetadataSchema</code> argument is
    * true a "metadata" schema containing definitions of tables, columns etc. is
    * added to root schema. */
-  public static OptiqRootSchema createRootSchema(boolean addMetadataSchema) {
-    OptiqRootSchema rootSchema =
-        new OptiqRootSchema(new OptiqConnectionImpl.RootSchema());
+  public static CalciteRootSchema createRootSchema(boolean addMetadataSchema) {
+    CalciteRootSchema rootSchema =
+        new CalciteRootSchema(new CalciteConnectionImpl.RootSchema());
     if (addMetadataSchema) {
       rootSchema.add("metadata", MetadataSchema.INSTANCE);
     }
@@ -157,10 +174,10 @@ public class OptiqSchema {
     return entry;
   }
 
-  public OptiqRootSchema root() {
-    for (OptiqSchema schema = this;;) {
+  public CalciteRootSchema root() {
+    for (CalciteSchema schema = this;;) {
       if (schema.parent == null) {
-        return (OptiqRootSchema) schema;
+        return (CalciteRootSchema) schema;
       }
       schema = schema.parent;
     }
@@ -172,7 +189,7 @@ public class OptiqSchema {
     if (name != null) {
       list.add(name);
     }
-    for (OptiqSchema s = this; s != null; s = s.parent) {
+    for (CalciteSchema s = this; s != null; s = s.parent) {
       if (s.parent != null || !s.name.equals("")) {
         // Omit the root schema's name from the path if it's the empty string,
         // which it usually is.
@@ -193,11 +210,11 @@ public class OptiqSchema {
     this.cache = cache;
   }
 
-  public final OptiqSchema getSubSchema(String schemaName,
+  public final CalciteSchema getSubSchema(String schemaName,
       boolean caseSensitive) {
     if (caseSensitive) {
       // Check explicit schemas, case-sensitive.
-      final OptiqSchema entry = subSchemaMap.get(schemaName);
+      final CalciteSchema entry = subSchemaMap.get(schemaName);
       if (entry != null) {
         return entry;
       }
@@ -211,7 +228,7 @@ public class OptiqSchema {
     } else {
       // Check explicit schemas, case-insensitive.
       //noinspection LoopStatementThatDoesntLoop
-      for (Map.Entry<String, OptiqSchema> entry
+      for (Map.Entry<String, CalciteSchema> entry
           : find(subSchemaMap, schemaName).entrySet()) {
         return entry.getValue();
       }
@@ -228,10 +245,10 @@ public class OptiqSchema {
   }
 
   /** Adds a child schema of this schema. */
-  public OptiqSchema add(String name, Schema schema) {
-    final OptiqSchema optiqSchema = new OptiqSchema(this, schema, name);
-    subSchemaMap.put(name, optiqSchema);
-    return optiqSchema;
+  public CalciteSchema add(String name, Schema schema) {
+    final CalciteSchema calciteSchema = new CalciteSchema(this, schema, name);
+    subSchemaMap.put(name, calciteSchema);
+    return calciteSchema;
   }
 
   /** Returns a table that materializes the given SQL statement. */
@@ -292,8 +309,8 @@ public class OptiqSchema {
     return new SchemaPlusImpl();
   }
 
-  public static OptiqSchema from(SchemaPlus plus) {
-    return ((SchemaPlusImpl) plus).optiqSchema();
+  public static CalciteSchema from(SchemaPlus plus) {
+    return ((SchemaPlusImpl) plus).calciteSchema();
   }
 
   /** Returns the default path resolving functions from this schema.
@@ -314,14 +331,14 @@ public class OptiqSchema {
   }
 
   /** Returns a collection of sub-schemas, both explicit (defined using
-   * {@link #add(String, net.hydromatic.optiq.Schema)}) and implicit
-   * (defined using {@link net.hydromatic.optiq.Schema#getSubSchemaNames()}
+   * {@link #add(String, org.apache.calcite.schema.Schema)}) and implicit
+   * (defined using {@link org.apache.calcite.schema.Schema#getSubSchemaNames()}
    * and {@link Schema#getSubSchema(String)}). */
-  public NavigableMap<String, OptiqSchema> getSubSchemaMap() {
+  public NavigableMap<String, CalciteSchema> getSubSchemaMap() {
     // Build a map of implicit sub-schemas first, then explicit sub-schemas.
     // If there are implicit and explicit with the same name, explicit wins.
-    final ImmutableSortedMap.Builder<String, OptiqSchema> builder =
-        new ImmutableSortedMap.Builder<String, OptiqSchema>(COMPARATOR);
+    final ImmutableSortedMap.Builder<String, CalciteSchema> builder =
+        new ImmutableSortedMap.Builder<String, CalciteSchema>(COMPARATOR);
     final long now = System.currentTimeMillis();
     final SubSchemaCache subSchemaCache = implicitSubSchemaCache.get(now);
     for (String name : subSchemaCache.names) {
@@ -501,10 +518,10 @@ public class OptiqSchema {
    * <p>The members of a schema must have unique names.
    */
   public abstract static class Entry {
-    public final OptiqSchema schema;
+    public final CalciteSchema schema;
     public final String name;
 
-    public Entry(OptiqSchema schema, String name) {
+    public Entry(CalciteSchema schema, String name) {
       Linq4j.requireNonNull(schema);
       Linq4j.requireNonNull(name);
       this.schema = schema;
@@ -521,7 +538,7 @@ public class OptiqSchema {
   public abstract static class TableEntry extends Entry {
     public final List<String> sqls;
 
-    public TableEntry(OptiqSchema schema, String name,
+    public TableEntry(CalciteSchema schema, String name,
         ImmutableList<String> sqls) {
       super(schema, name);
       this.sqls = Preconditions.checkNotNull(sqls);
@@ -532,7 +549,7 @@ public class OptiqSchema {
 
   /** Membership of a function in a schema. */
   public abstract static class FunctionEntry extends Entry {
-    public FunctionEntry(OptiqSchema schema, String name) {
+    public FunctionEntry(CalciteSchema schema, String name) {
       super(schema, name);
     }
 
@@ -545,7 +562,7 @@ public class OptiqSchema {
 
   /** Membership of a lattice in a schema. */
   public abstract static class LatticeEntry extends Entry {
-    public LatticeEntry(OptiqSchema schema, String name) {
+    public LatticeEntry(CalciteSchema schema, String name) {
       super(schema, name);
     }
 
@@ -554,10 +571,11 @@ public class OptiqSchema {
     public abstract TableEntry getStarTable();
   }
 
-  /** Implementation of {@link SchemaPlus} based on an {@code OptiqSchema}. */
+  /** Implementation of {@link SchemaPlus} based on a
+   * {@link org.apache.calcite.jdbc.CalciteSchema}. */
   private class SchemaPlusImpl implements SchemaPlus {
-    public OptiqSchema optiqSchema() {
-      return OptiqSchema.this;
+    CalciteSchema calciteSchema() {
+      return CalciteSchema.this;
     }
 
     public SchemaPlus getParentSchema() {
@@ -565,7 +583,7 @@ public class OptiqSchema {
     }
 
     public String getName() {
-      return OptiqSchema.this.getName();
+      return CalciteSchema.this.getName();
     }
 
     public boolean isMutable() {
@@ -573,11 +591,11 @@ public class OptiqSchema {
     }
 
     public void setCacheEnabled(boolean cache) {
-      OptiqSchema.this.setCache(cache);
+      CalciteSchema.this.setCache(cache);
     }
 
     public boolean isCacheEnabled() {
-      return OptiqSchema.this.cache;
+      return CalciteSchema.this.cache;
     }
 
     public boolean contentsHaveChangedSince(long lastCheck, long now) {
@@ -589,75 +607,76 @@ public class OptiqSchema {
     }
 
     public Table getTable(String name) {
-      final Pair<String, Table> pair = OptiqSchema.this.getTable(name, true);
+      final Pair<String, Table> pair = CalciteSchema.this.getTable(name, true);
       return pair == null ? null : pair.getValue();
     }
 
     public NavigableSet<String> getTableNames() {
-      return OptiqSchema.this.getTableNames();
+      return CalciteSchema.this.getTableNames();
     }
 
     public Collection<Function> getFunctions(String name) {
-      return OptiqSchema.this.getFunctions(name, true);
+      return CalciteSchema.this.getFunctions(name, true);
     }
 
     public NavigableSet<String> getFunctionNames() {
-      return OptiqSchema.this.getFunctionNames();
+      return CalciteSchema.this.getFunctionNames();
     }
 
     public SchemaPlus getSubSchema(String name) {
-      final OptiqSchema subSchema = OptiqSchema.this.getSubSchema(name, true);
+      final CalciteSchema subSchema =
+          CalciteSchema.this.getSubSchema(name, true);
       return subSchema == null ? null : subSchema.plus();
     }
 
     public Set<String> getSubSchemaNames() {
-      return OptiqSchema.this.getSubSchemaMap().keySet();
+      return CalciteSchema.this.getSubSchemaMap().keySet();
     }
 
     public SchemaPlus add(String name, Schema schema) {
-      final OptiqSchema optiqSchema = OptiqSchema.this.add(name, schema);
-      return optiqSchema.plus();
+      final CalciteSchema calciteSchema = CalciteSchema.this.add(name, schema);
+      return calciteSchema.plus();
     }
 
     public <T> T unwrap(Class<T> clazz) {
       if (clazz.isInstance(this)) {
         return clazz.cast(this);
       }
-      if (clazz.isInstance(OptiqSchema.this)) {
-        return clazz.cast(OptiqSchema.this);
+      if (clazz.isInstance(CalciteSchema.this)) {
+        return clazz.cast(CalciteSchema.this);
       }
-      if (clazz.isInstance(OptiqSchema.this.schema)) {
-        return clazz.cast(OptiqSchema.this.schema);
+      if (clazz.isInstance(CalciteSchema.this.schema)) {
+        return clazz.cast(CalciteSchema.this.schema);
       }
       throw new ClassCastException("not a " + clazz);
     }
 
     public void setPath(ImmutableList<ImmutableList<String>> path) {
-      OptiqSchema.this.path = path;
+      CalciteSchema.this.path = path;
     }
 
     public void add(String name, Table table) {
-      OptiqSchema.this.add(name, table);
+      CalciteSchema.this.add(name, table);
     }
 
-    public void add(String name, net.hydromatic.optiq.Function function) {
-      OptiqSchema.this.add(name, function);
+    public void add(String name, Function function) {
+      CalciteSchema.this.add(name, function);
     }
 
     public void add(String name, Lattice lattice) {
-      OptiqSchema.this.add(name, lattice);
+      CalciteSchema.this.add(name, lattice);
     }
   }
 
   /**
-   * Implementation of {@link net.hydromatic.optiq.jdbc.OptiqSchema.TableEntry}
+   * Implementation of {@link CalciteSchema.TableEntry}
    * where all properties are held in fields.
    */
   public static class TableEntryImpl extends TableEntry {
     private final Table table;
 
     /** Creates a TableEntryImpl. */
-    public TableEntryImpl(OptiqSchema schema, String name, Table table,
+    public TableEntryImpl(CalciteSchema schema, String name, Table table,
         ImmutableList<String> sqls) {
       super(schema, name, sqls);
       assert table != null;
@@ -677,7 +696,7 @@ public class OptiqSchema {
     private final Function function;
 
     /** Creates a FunctionEntryImpl. */
-    public FunctionEntryImpl(OptiqSchema schema, String name,
+    public FunctionEntryImpl(CalciteSchema schema, String name,
         Function function) {
       super(schema, name);
       this.function = function;
@@ -699,10 +718,11 @@ public class OptiqSchema {
    */
   public static class LatticeEntryImpl extends LatticeEntry {
     private final Lattice lattice;
-    private final OptiqSchema.TableEntry starTableEntry;
+    private final CalciteSchema.TableEntry starTableEntry;
 
     /** Creates a LatticeEntryImpl. */
-    public LatticeEntryImpl(OptiqSchema schema, String name, Lattice lattice) {
+    public LatticeEntryImpl(CalciteSchema schema, String name,
+        Lattice lattice) {
       super(schema, name);
       this.lattice = lattice;
 
@@ -732,18 +752,18 @@ public class OptiqSchema {
     /** Creates a new value. */
     T build();
 
-    /** Called when OptiqSchema caching is enabled or disabled. */
+    /** Called when CalciteSchema caching is enabled or disabled. */
     void enable(long now, boolean enabled);
   }
 
-  /** Implementation of {@link net.hydromatic.optiq.jdbc.OptiqSchema.Cached}
-   * that drives from {@link net.hydromatic.optiq.jdbc.OptiqSchema#cache}. */
+  /** Implementation of {@link CalciteSchema.Cached}
+   * that drives from {@link CalciteSchema#cache}. */
   private abstract class AbstractCached<T> implements Cached<T> {
     T t;
     long checked = Long.MIN_VALUE;
 
     public T get(long now) {
-      if (!OptiqSchema.this.cache) {
+      if (!CalciteSchema.this.cache) {
         return build();
       }
       if (checked == Long.MIN_VALUE
@@ -762,32 +782,32 @@ public class OptiqSchema {
     }
   }
 
-  /** Information about the implicit sub-schemas of an {@link OptiqSchema}. */
+  /** Information about the implicit sub-schemas of an {@link CalciteSchema}. */
   private static class SubSchemaCache {
     /** The names of sub-schemas returned from the {@link Schema} SPI. */
     final NavigableSet<String> names;
-    /** Cached {@link net.hydromatic.optiq.jdbc.OptiqSchema} wrappers. It is
+    /** Cached {@link CalciteSchema} wrappers. It is
      * worth caching them because they contain maps of their own sub-objects. */
-    final LoadingCache<String, OptiqSchema> cache;
+    final LoadingCache<String, CalciteSchema> cache;
 
-    private SubSchemaCache(final OptiqSchema optiqSchema,
+    private SubSchemaCache(final CalciteSchema calciteSchema,
         NavigableSet<String> names) {
       this.names = names;
       this.cache = CacheBuilder.newBuilder().build(
-          new CacheLoader<String, OptiqSchema>() {
+          new CacheLoader<String, CalciteSchema>() {
             @SuppressWarnings("NullableProblems")
-            @Override public OptiqSchema load(String schemaName) {
+            @Override public CalciteSchema load(String schemaName) {
               final Schema subSchema =
-                  optiqSchema.schema.getSubSchema(schemaName);
+                  calciteSchema.schema.getSubSchema(schemaName);
               if (subSchema == null) {
                 throw new RuntimeException("sub-schema " + schemaName
                     + " not found");
               }
-              return new OptiqSchema(optiqSchema, subSchema, schemaName);
+              return new CalciteSchema(calciteSchema, subSchema, schemaName);
             }
           });
     }
   }
 }
 
-// End OptiqSchema.java
+// End CalciteSchema.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/jdbc/CalciteStatement.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/jdbc/CalciteStatement.java b/core/src/main/java/org/apache/calcite/jdbc/CalciteStatement.java
index eb29453..c8f0e17 100644
--- a/core/src/main/java/org/apache/calcite/jdbc/CalciteStatement.java
+++ b/core/src/main/java/org/apache/calcite/jdbc/CalciteStatement.java
@@ -14,23 +14,22 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.jdbc;
+package org.apache.calcite.jdbc;
 
-import net.hydromatic.avatica.*;
-
-import net.hydromatic.linq4j.Queryable;
-
-import net.hydromatic.optiq.server.OptiqServerStatement;
+import org.apache.calcite.avatica.AvaticaResultSet;
+import org.apache.calcite.avatica.AvaticaStatement;
+import org.apache.calcite.linq4j.Queryable;
+import org.apache.calcite.server.CalciteServerStatement;
 
 /**
  * Implementation of {@link java.sql.Statement}
  * for the Calcite engine.
  */
-public abstract class OptiqStatement
+public abstract class CalciteStatement
     extends AvaticaStatement
-    implements OptiqServerStatement {
-  OptiqStatement(
-      OptiqConnectionImpl connection,
+    implements CalciteServerStatement {
+  CalciteStatement(
+      CalciteConnectionImpl connection,
       int resultSetType,
       int resultSetConcurrency,
       int resultSetHoldability) {
@@ -40,24 +39,25 @@ public abstract class OptiqStatement
 
   // implement Statement
 
-  @Override public OptiqConnectionImpl getConnection() {
-    return (OptiqConnectionImpl) connection;
+  @Override public CalciteConnectionImpl getConnection() {
+    return (CalciteConnectionImpl) connection;
   }
 
-  public OptiqConnectionImpl.ContextImpl createPrepareContext() {
-    return new OptiqConnectionImpl.ContextImpl(getConnection());
+  public CalciteConnectionImpl.ContextImpl createPrepareContext() {
+    return new CalciteConnectionImpl.ContextImpl(getConnection());
   }
 
-  protected <T> OptiqPrepare.PrepareResult<T> prepare(Queryable<T> queryable) {
-    final OptiqPrepare prepare = getConnection().prepareFactory.apply();
+  protected <T> CalcitePrepare.PrepareResult<T> prepare(
+      Queryable<T> queryable) {
+    final CalcitePrepare prepare = getConnection().prepareFactory.apply();
     return prepare.prepareQueryable(createPrepareContext(), queryable);
   }
 
-  @Override
-  protected void close_() {
+  @Override protected void close_() {
     if (!closed) {
       closed = true;
-      final OptiqConnectionImpl connection1 = (OptiqConnectionImpl) connection;
+      final CalciteConnectionImpl connection1 =
+          (CalciteConnectionImpl) connection;
       connection1.server.removeStatement(this);
       if (openResultSet != null) {
         AvaticaResultSet c = openResultSet;
@@ -71,4 +71,4 @@ public abstract class OptiqStatement
   }
 }
 
-// End OptiqStatement.java
+// End CalciteStatement.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/jdbc/Driver.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/jdbc/Driver.java b/core/src/main/java/org/apache/calcite/jdbc/Driver.java
index a6a25b3..5997f22 100644
--- a/core/src/main/java/org/apache/calcite/jdbc/Driver.java
+++ b/core/src/main/java/org/apache/calcite/jdbc/Driver.java
@@ -14,19 +14,27 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.jdbc;
-
-import net.hydromatic.avatica.*;
-
-import net.hydromatic.linq4j.function.Function0;
-
-import net.hydromatic.optiq.config.OptiqConnectionProperty;
-import net.hydromatic.optiq.impl.java.JavaTypeFactory;
-import net.hydromatic.optiq.model.ModelHandler;
+package org.apache.calcite.jdbc;
+
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.avatica.AvaticaConnection;
+import org.apache.calcite.avatica.BuiltInConnectionProperty;
+import org.apache.calcite.avatica.ConnectionProperty;
+import org.apache.calcite.avatica.DriverVersion;
+import org.apache.calcite.avatica.Handler;
+import org.apache.calcite.avatica.HandlerImpl;
+import org.apache.calcite.avatica.UnregisteredDriver;
+import org.apache.calcite.config.CalciteConnectionProperty;
+import org.apache.calcite.linq4j.function.Function0;
+import org.apache.calcite.model.ModelHandler;
 
 import java.io.IOException;
 import java.sql.SQLException;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Properties;
 
 /**
  * Calcite JDBC driver.
@@ -34,7 +42,7 @@ import java.util.*;
 public class Driver extends UnregisteredDriver {
   public static final String CONNECT_STRING_PREFIX = "jdbc:calcite:";
 
-  final Function0<OptiqPrepare> prepareFactory;
+  final Function0<CalcitePrepare> prepareFactory;
 
   static {
     new Driver().register();
@@ -45,46 +53,42 @@ public class Driver extends UnregisteredDriver {
     this.prepareFactory = createPrepareFactory();
   }
 
-  protected Function0<OptiqPrepare> createPrepareFactory() {
-    return OptiqPrepare.DEFAULT_FACTORY;
+  protected Function0<CalcitePrepare> createPrepareFactory() {
+    return CalcitePrepare.DEFAULT_FACTORY;
   }
 
-  @Override
-  protected String getConnectStringPrefix() {
+  @Override protected String getConnectStringPrefix() {
     return CONNECT_STRING_PREFIX;
   }
 
-  @Override
-  protected String getFactoryClassName(JdbcVersion jdbcVersion) {
+  @Override protected String getFactoryClassName(JdbcVersion jdbcVersion) {
     switch (jdbcVersion) {
     case JDBC_30:
-      return "net.hydromatic.optiq.jdbc.OptiqJdbc3Factory";
+      return "org.apache.calcite.jdbc.CalciteJdbc3Factory";
     case JDBC_40:
-      return "net.hydromatic.optiq.jdbc.OptiqJdbc40Factory";
+      return "org.apache.calcite.jdbc.CalciteJdbc40Factory";
     case JDBC_41:
     default:
-      return "net.hydromatic.optiq.jdbc.OptiqJdbc41Factory";
+      return "org.apache.calcite.jdbc.CalciteJdbc41Factory";
     }
   }
 
   protected DriverVersion createDriverVersion() {
     return DriverVersion.load(
         Driver.class,
-        "net-hydromatic-optiq-jdbc.properties",
+        "org-apache-calcite-jdbc.properties",
         "Calcite JDBC Driver",
         "unknown version",
         "Calcite",
         "unknown version");
   }
 
-  @Override
-  protected Handler createHandler() {
+  @Override protected Handler createHandler() {
     return new HandlerImpl() {
-      @Override
-      public void onConnectionInit(AvaticaConnection connection_)
+      @Override public void onConnectionInit(AvaticaConnection connection_)
           throws SQLException {
-        final OptiqConnectionImpl connection =
-            (OptiqConnectionImpl) connection_;
+        final CalciteConnectionImpl connection =
+            (CalciteConnectionImpl) connection_;
         super.onConnectionInit(connection);
         final String model = connection.config().model();
         if (model != null) {
@@ -99,18 +103,17 @@ public class Driver extends UnregisteredDriver {
     };
   }
 
-  @Override
-  protected Collection<ConnectionProperty> getConnectionProperties() {
+  @Override protected Collection<ConnectionProperty> getConnectionProperties() {
     final List<ConnectionProperty> list = new ArrayList<ConnectionProperty>();
     Collections.addAll(list, BuiltInConnectionProperty.values());
-    Collections.addAll(list, OptiqConnectionProperty.values());
+    Collections.addAll(list, CalciteConnectionProperty.values());
     return list;
   }
 
   /** Creates an internal connection. */
-  OptiqConnection connect(OptiqRootSchema rootSchema,
+  CalciteConnection connect(CalciteRootSchema rootSchema,
       JavaTypeFactory typeFactory) {
-    return (OptiqConnection) ((OptiqFactory) factory)
+    return (CalciteConnection) ((CalciteFactory) factory)
         .newConnection(this, factory, CONNECT_STRING_PREFIX, new Properties(),
             rootSchema, typeFactory);
   }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/jdbc/JavaRecordType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/jdbc/JavaRecordType.java b/core/src/main/java/org/apache/calcite/jdbc/JavaRecordType.java
index 0e77a60..350a3ef 100644
--- a/core/src/main/java/org/apache/calcite/jdbc/JavaRecordType.java
+++ b/core/src/main/java/org/apache/calcite/jdbc/JavaRecordType.java
@@ -14,10 +14,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.jdbc;
+package org.apache.calcite.jdbc;
 
-import org.eigenbase.reltype.RelDataTypeField;
-import org.eigenbase.reltype.RelRecordType;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rel.type.RelRecordType;
 
 import java.util.List;
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/jdbc/JavaTypeFactoryImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/jdbc/JavaTypeFactoryImpl.java b/core/src/main/java/org/apache/calcite/jdbc/JavaTypeFactoryImpl.java
index 6d173f6..3f3fabb 100644
--- a/core/src/main/java/org/apache/calcite/jdbc/JavaTypeFactoryImpl.java
+++ b/core/src/main/java/org/apache/calcite/jdbc/JavaTypeFactoryImpl.java
@@ -14,21 +14,26 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.jdbc;
+package org.apache.calcite.jdbc;
 
-import net.hydromatic.avatica.ByteString;
-
-import net.hydromatic.linq4j.Ord;
-import net.hydromatic.linq4j.expressions.Primitive;
-import net.hydromatic.linq4j.expressions.Types;
-
-import net.hydromatic.optiq.impl.java.JavaTypeFactory;
-import net.hydromatic.optiq.runtime.Unit;
-
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.util.Pair;
-import org.eigenbase.util.Util;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.avatica.ByteString;
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.linq4j.tree.Primitive;
+import org.apache.calcite.linq4j.tree.Types;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rel.type.RelDataTypeFieldImpl;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.rel.type.RelRecordType;
+import org.apache.calcite.runtime.Unit;
+import org.apache.calcite.sql.type.BasicSqlType;
+import org.apache.calcite.sql.type.IntervalSqlType;
+import org.apache.calcite.sql.type.JavaToSqlTypeConversionRules;
+import org.apache.calcite.sql.type.SqlTypeFactoryImpl;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
 
 import com.google.common.base.Function;
 import com.google.common.collect.Lists;
@@ -38,7 +43,11 @@ import java.lang.reflect.Modifier;
 import java.lang.reflect.Type;
 import java.math.BigDecimal;
 import java.sql.Array;
-import java.util.*;
+import java.util.AbstractList;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
 
 /**
  * Implementation of {@link JavaTypeFactory}.


[05/58] [abbrv] [partial] incubator-calcite git commit: [CALCITE-306] Standardize code style for "import package.*; "

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/resources/sql/misc.oq
----------------------------------------------------------------------
diff --git a/core/src/test/resources/sql/misc.oq b/core/src/test/resources/sql/misc.oq
index e68e65b..5af6854 100644
--- a/core/src/test/resources/sql/misc.oq
+++ b/core/src/test/resources/sql/misc.oq
@@ -167,12 +167,12 @@ and e."name" <> d."name";
 (3 rows)
 
 !ok
-EnumerableCalcRel(expr#0..4=[{inputs}], expr#5=[CAST($t2):VARCHAR(1) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary"], expr#6=[CAST($t4):VARCHAR(1) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary"], expr#7=[<>($t5, $t6)], empid=[$t0], name=[$t4], name0=[$t2], $condition=[$t7])
-  EnumerableJoinRel(condition=[=($1, $3)], joinType=[inner])
-    EnumerableCalcRel(expr#0..4=[{inputs}], proj#0..2=[{exprs}])
-      EnumerableTableAccessRel(table=[[hr, emps]])
-    EnumerableCalcRel(expr#0..2=[{inputs}], proj#0..1=[{exprs}])
-      EnumerableTableAccessRel(table=[[hr, depts]])
+EnumerableCalc(expr#0..4=[{inputs}], expr#5=[CAST($t2):VARCHAR(1) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary"], expr#6=[CAST($t4):VARCHAR(1) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary"], expr#7=[<>($t5, $t6)], empid=[$t0], name=[$t4], name0=[$t2], $condition=[$t7])
+  EnumerableJoin(condition=[=($1, $3)], joinType=[inner])
+    EnumerableCalc(expr#0..4=[{inputs}], proj#0..2=[{exprs}])
+      EnumerableTableScan(table=[[hr, emps]])
+    EnumerableCalc(expr#0..2=[{inputs}], proj#0..1=[{exprs}])
+      EnumerableTableScan(table=[[hr, depts]])
 !plan
 
 # Same query, expressed using WHERE.
@@ -191,12 +191,12 @@ and e."name" <> d."name";
 (3 rows)
 
 !ok
-EnumerableCalcRel(expr#0..4=[{inputs}], expr#5=[CAST($t2):VARCHAR(1) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary"], expr#6=[CAST($t4):VARCHAR(1) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary"], expr#7=[<>($t5, $t6)], empid=[$t0], name=[$t4], name0=[$t2], $condition=[$t7])
-  EnumerableJoinRel(condition=[=($1, $3)], joinType=[inner])
-    EnumerableCalcRel(expr#0..4=[{inputs}], proj#0..2=[{exprs}])
-      EnumerableTableAccessRel(table=[[hr, emps]])
-    EnumerableCalcRel(expr#0..2=[{inputs}], proj#0..1=[{exprs}])
-      EnumerableTableAccessRel(table=[[hr, depts]])
+EnumerableCalc(expr#0..4=[{inputs}], expr#5=[CAST($t2):VARCHAR(1) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary"], expr#6=[CAST($t4):VARCHAR(1) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary"], expr#7=[<>($t5, $t6)], empid=[$t0], name=[$t4], name0=[$t2], $condition=[$t7])
+  EnumerableJoin(condition=[=($1, $3)], joinType=[inner])
+    EnumerableCalc(expr#0..4=[{inputs}], proj#0..2=[{exprs}])
+      EnumerableTableScan(table=[[hr, emps]])
+    EnumerableCalc(expr#0..2=[{inputs}], proj#0..1=[{exprs}])
+      EnumerableTableScan(table=[[hr, depts]])
 !plan
 
 # EXISTS
@@ -213,14 +213,14 @@ where exists (
 (3 rows)
 
 !ok
-EnumerableSemiJoinRel(condition=[=($1, $5)], joinType=[inner])
-  EnumerableTableAccessRel(table=[[hr, emps]])
-  EnumerableCalcRel(expr#0..3=[{inputs}], expr#4=[true], $f01=[$t0], $f0=[$t4])
-    EnumerableJoinRel(condition=[=($0, $1)], joinType=[inner])
-      EnumerableAggregateRel(group=[{0}])
-        EnumerableCalcRel(expr#0..4=[{inputs}], $f0=[$t1])
-          EnumerableTableAccessRel(table=[[hr, emps]])
-      EnumerableTableAccessRel(table=[[hr, depts]])
+EnumerableSemiJoin(condition=[=($1, $5)], joinType=[inner])
+  EnumerableTableScan(table=[[hr, emps]])
+  EnumerableCalc(expr#0..3=[{inputs}], expr#4=[true], $f01=[$t0], $f0=[$t4])
+    EnumerableJoin(condition=[=($0, $1)], joinType=[inner])
+      EnumerableAggregate(group=[{0}])
+        EnumerableCalc(expr#0..4=[{inputs}], $f0=[$t1])
+          EnumerableTableScan(table=[[hr, emps]])
+      EnumerableTableScan(table=[[hr, depts]])
 !plan
 
 # NOT EXISTS
@@ -236,16 +236,16 @@ where not exists (
 (1 row)
 
 !ok
-EnumerableCalcRel(expr#0..6=[{inputs}], expr#7=[IS NOT NULL($t6)], expr#8=[NOT($t7)], proj#0..4=[{exprs}], $condition=[$t8])
-  EnumerableJoinRel(condition=[=($1, $5)], joinType=[left])
-    EnumerableTableAccessRel(table=[[hr, emps]])
-    EnumerableAggregateRel(group=[{0}], agg#0=[MIN($1)])
-      EnumerableCalcRel(expr#0..3=[{inputs}], expr#4=[true], $f01=[$t0], $f0=[$t4])
-        EnumerableJoinRel(condition=[=($0, $1)], joinType=[inner])
-          EnumerableAggregateRel(group=[{0}])
-            EnumerableCalcRel(expr#0..4=[{inputs}], $f0=[$t1])
-              EnumerableTableAccessRel(table=[[hr, emps]])
-          EnumerableTableAccessRel(table=[[hr, depts]])
+EnumerableCalc(expr#0..6=[{inputs}], expr#7=[IS NOT NULL($t6)], expr#8=[NOT($t7)], proj#0..4=[{exprs}], $condition=[$t8])
+  EnumerableJoin(condition=[=($1, $5)], joinType=[left])
+    EnumerableTableScan(table=[[hr, emps]])
+    EnumerableAggregate(group=[{0}], agg#0=[MIN($1)])
+      EnumerableCalc(expr#0..3=[{inputs}], expr#4=[true], $f01=[$t0], $f0=[$t4])
+        EnumerableJoin(condition=[=($0, $1)], joinType=[inner])
+          EnumerableAggregate(group=[{0}])
+            EnumerableCalc(expr#0..4=[{inputs}], $f0=[$t1])
+              EnumerableTableScan(table=[[hr, emps]])
+          EnumerableTableScan(table=[[hr, depts]])
 !plan
 
 # NOT EXISTS .. OR NOT EXISTS
@@ -266,32 +266,32 @@ or not exists (
 (3 rows)
 
 !ok
-EnumerableCalcRel(expr#0..7=[{inputs}], expr#8=[IS NOT NULL($t5)], expr#9=[NOT($t8)], expr#10=[IS NOT NULL($t7)], expr#11=[NOT($t10)], expr#12=[OR($t9, $t11)], proj#0..4=[{exprs}], $condition=[$t12])
-  EnumerableJoinRel(condition=[=($0, $6)], joinType=[left])
-    EnumerableCalcRel(expr#0..6=[{inputs}], proj#0..4=[{exprs}], $f0=[$t6])
-      EnumerableJoinRel(condition=[=($1, $5)], joinType=[left])
-        EnumerableTableAccessRel(table=[[hr, emps]])
-        EnumerableAggregateRel(group=[{0}], agg#0=[MIN($1)])
-          EnumerableCalcRel(expr#0..3=[{inputs}], expr#4=[true], $f01=[$t0], $f0=[$t4])
-            EnumerableJoinRel(condition=[=($0, $1)], joinType=[inner])
-              EnumerableAggregateRel(group=[{0}])
-                EnumerableCalcRel(expr#0..4=[{inputs}], $f0=[$t1])
-                  EnumerableTableAccessRel(table=[[hr, emps]])
-              EnumerableTableAccessRel(table=[[hr, depts]])
-    EnumerableAggregateRel(group=[{0}], agg#0=[MIN($1)])
-      EnumerableCalcRel(expr#0..3=[{inputs}], expr#4=[true], expr#5=[90], expr#6=[+($t1, $t5)], expr#7=[CAST($t0):INTEGER NOT NULL], expr#8=[=($t6, $t7)], $f01=[$t0], $f0=[$t4], $condition=[$t8])
-        EnumerableJoinRel(condition=[true], joinType=[inner])
-          EnumerableAggregateRel(group=[{0}])
-            EnumerableCalcRel(expr#0..4=[{inputs}], $f0=[$t0])
-              EnumerableSemiJoinRel(condition=[=($1, $5)], joinType=[inner])
-                EnumerableTableAccessRel(table=[[hr, emps]])
-                EnumerableCalcRel(expr#0..3=[{inputs}], expr#4=[true], $f01=[$t0], $f0=[$t4])
-                  EnumerableJoinRel(condition=[=($0, $1)], joinType=[inner])
-                    EnumerableAggregateRel(group=[{0}])
-                      EnumerableCalcRel(expr#0..4=[{inputs}], $f0=[$t1])
-                        EnumerableTableAccessRel(table=[[hr, emps]])
-                    EnumerableTableAccessRel(table=[[hr, depts]])
-          EnumerableTableAccessRel(table=[[hr, depts]])
+EnumerableCalc(expr#0..7=[{inputs}], expr#8=[IS NOT NULL($t5)], expr#9=[NOT($t8)], expr#10=[IS NOT NULL($t7)], expr#11=[NOT($t10)], expr#12=[OR($t9, $t11)], proj#0..4=[{exprs}], $condition=[$t12])
+  EnumerableJoin(condition=[=($0, $6)], joinType=[left])
+    EnumerableCalc(expr#0..6=[{inputs}], proj#0..4=[{exprs}], $f0=[$t6])
+      EnumerableJoin(condition=[=($1, $5)], joinType=[left])
+        EnumerableTableScan(table=[[hr, emps]])
+        EnumerableAggregate(group=[{0}], agg#0=[MIN($1)])
+          EnumerableCalc(expr#0..3=[{inputs}], expr#4=[true], $f01=[$t0], $f0=[$t4])
+            EnumerableJoin(condition=[=($0, $1)], joinType=[inner])
+              EnumerableAggregate(group=[{0}])
+                EnumerableCalc(expr#0..4=[{inputs}], $f0=[$t1])
+                  EnumerableTableScan(table=[[hr, emps]])
+              EnumerableTableScan(table=[[hr, depts]])
+    EnumerableAggregate(group=[{0}], agg#0=[MIN($1)])
+      EnumerableCalc(expr#0..3=[{inputs}], expr#4=[true], expr#5=[90], expr#6=[+($t1, $t5)], expr#7=[CAST($t0):INTEGER NOT NULL], expr#8=[=($t6, $t7)], $f01=[$t0], $f0=[$t4], $condition=[$t8])
+        EnumerableJoin(condition=[true], joinType=[inner])
+          EnumerableAggregate(group=[{0}])
+            EnumerableCalc(expr#0..4=[{inputs}], $f0=[$t0])
+              EnumerableSemiJoin(condition=[=($1, $5)], joinType=[inner])
+                EnumerableTableScan(table=[[hr, emps]])
+                EnumerableCalc(expr#0..3=[{inputs}], expr#4=[true], $f01=[$t0], $f0=[$t4])
+                  EnumerableJoin(condition=[=($0, $1)], joinType=[inner])
+                    EnumerableAggregate(group=[{0}])
+                      EnumerableCalc(expr#0..4=[{inputs}], $f0=[$t1])
+                        EnumerableTableScan(table=[[hr, emps]])
+                    EnumerableTableScan(table=[[hr, depts]])
+          EnumerableTableScan(table=[[hr, depts]])
 !plan
 
 # Filter combined with an OR filter.
@@ -338,8 +338,8 @@ select 1 from "hr"."emps";
 (4 rows)
 
 !ok
-EnumerableCalcRel(expr#0..4=[{inputs}], expr#5=[1], EXPR$0=[$t5])
-  EnumerableTableAccessRel(table=[[hr, emps]])
+EnumerableCalc(expr#0..4=[{inputs}], expr#5=[1], EXPR$0=[$t5])
+  EnumerableTableScan(table=[[hr, emps]])
 !plan
 
 # [CALCITE-393] for table scan under join
@@ -352,13 +352,13 @@ select count(*) as c from "hr"."emps", "hr"."depts";
 (1 row)
 
 !ok
-EnumerableAggregateRel(group=[{}], C=[COUNT()])
-  EnumerableCalcRel(expr#0..1=[{inputs}], expr#2=[0], DUMMY=[$t2])
-    EnumerableJoinRel(condition=[true], joinType=[inner])
-      EnumerableCalcRel(expr#0..4=[{inputs}], expr#5=[0], DUMMY=[$t5])
-        EnumerableTableAccessRel(table=[[hr, emps]])
-      EnumerableCalcRel(expr#0..2=[{inputs}], expr#3=[0], DUMMY=[$t3])
-        EnumerableTableAccessRel(table=[[hr, depts]])
+EnumerableAggregate(group=[{}], C=[COUNT()])
+  EnumerableCalc(expr#0..1=[{inputs}], expr#2=[0], DUMMY=[$t2])
+    EnumerableJoin(condition=[true], joinType=[inner])
+      EnumerableCalc(expr#0..4=[{inputs}], expr#5=[0], DUMMY=[$t5])
+        EnumerableTableScan(table=[[hr, emps]])
+      EnumerableCalc(expr#0..2=[{inputs}], expr#3=[0], DUMMY=[$t3])
+        EnumerableTableScan(table=[[hr, depts]])
 !plan
 
 # [CALCITE-345] AssertionError in RexToLixTranslator comparing to date literal
@@ -428,12 +428,12 @@ from "sales_fact_1997" as s
   join "customer" as c using ("customer_id")
   join "product" as p using ("product_id")
 where c."city" = 'San Francisco';
-EnumerableJoinRel(condition=[=($0, $38)], joinType=[inner])
-  EnumerableJoinRel(condition=[=($2, $8)], joinType=[inner])
-    EnumerableTableAccessRel(table=[[foodmart2, sales_fact_1997]])
-    EnumerableCalcRel(expr#0..28=[{inputs}], expr#29=['San Francisco'], expr#30=[=($t9, $t29)], proj#0..28=[{exprs}], $condition=[$t30])
-      EnumerableTableAccessRel(table=[[foodmart2, customer]])
-  EnumerableTableAccessRel(table=[[foodmart2, product]])
+EnumerableJoin(condition=[=($0, $38)], joinType=[inner])
+  EnumerableJoin(condition=[=($2, $8)], joinType=[inner])
+    EnumerableTableScan(table=[[foodmart2, sales_fact_1997]])
+    EnumerableCalc(expr#0..28=[{inputs}], expr#29=['San Francisco'], expr#30=[=($t9, $t29)], proj#0..28=[{exprs}], $condition=[$t30])
+      EnumerableTableScan(table=[[foodmart2, customer]])
+  EnumerableTableScan(table=[[foodmart2, product]])
 !plan
 
 # 4-way join whose optimal plan requires bushy join.
@@ -451,16 +451,16 @@ from "sales_fact_1997" as s
   join "product_class" as pc using ("product_class_id")
 where c."city" = 'San Francisco'
  and pc."product_department" = 'Snacks';
-EnumerableCalcRel(expr#0..56=[{inputs}], $f0=[$t20], $f1=[$t21], $f2=[$t22], $f3=[$t23], $f4=[$t24], $f5=[$t25], $f6=[$t26], $f7=[$t27], $f8=[$t28], $f9=[$t29], $f10=[$t30], $f11=[$t31], $f12=[$t32], $f13=[$t33], $f14=[$t34], $f15=[$t35], $f16=[$t36], $f17=[$t37], $f18=[$t38], $f19=[$t39], $f20=[$t40], $f21=[$t41], $f22=[$t42], $f23=[$t43], $f24=[$t44], $f25=[$t45], $f26=[$t46], $f27=[$t47], $f28=[$t48], $f29=[$t49], $f30=[$t50], $f31=[$t51], $f32=[$t52], $f33=[$t53], $f34=[$t54], $f35=[$t55], $f36=[$t56], $f37=[$t5], $f38=[$t6], $f39=[$t7], $f40=[$t8], $f41=[$t9], $f42=[$t10], $f43=[$t11], $f44=[$t12], $f45=[$t13], $f46=[$t14], $f47=[$t15], $f48=[$t16], $f49=[$t17], $f50=[$t18], $f51=[$t19], $f52=[$t0], $f53=[$t1], $f54=[$t2], $f55=[$t3], $f56=[$t4])
-  EnumerableJoinRel(condition=[=($6, $20)], joinType=[inner])
-    EnumerableJoinRel(condition=[=($0, $5)], joinType=[inner])
-      EnumerableCalcRel(expr#0..4=[{inputs}], expr#5=['Snacks'], expr#6=[=($t3, $t5)], proj#0..4=[{exprs}], $condition=[$t6])
-        EnumerableTableAccessRel(table=[[foodmart2, product_class]])
-      EnumerableTableAccessRel(table=[[foodmart2, product]])
-    EnumerableJoinRel(condition=[=($2, $8)], joinType=[inner])
-      EnumerableTableAccessRel(table=[[foodmart2, sales_fact_1997]])
-      EnumerableCalcRel(expr#0..28=[{inputs}], expr#29=['San Francisco'], expr#30=[=($t9, $t29)], proj#0..28=[{exprs}], $condition=[$t30])
-        EnumerableTableAccessRel(table=[[foodmart2, customer]])
+EnumerableCalc(expr#0..56=[{inputs}], $f0=[$t20], $f1=[$t21], $f2=[$t22], $f3=[$t23], $f4=[$t24], $f5=[$t25], $f6=[$t26], $f7=[$t27], $f8=[$t28], $f9=[$t29], $f10=[$t30], $f11=[$t31], $f12=[$t32], $f13=[$t33], $f14=[$t34], $f15=[$t35], $f16=[$t36], $f17=[$t37], $f18=[$t38], $f19=[$t39], $f20=[$t40], $f21=[$t41], $f22=[$t42], $f23=[$t43], $f24=[$t44], $f25=[$t45], $f26=[$t46], $f27=[$t47], $f28=[$t48], $f29=[$t49], $f30=[$t50], $f31=[$t51], $f32=[$t52], $f33=[$t53], $f34=[$t54], $f35=[$t55], $f36=[$t56], $f37=[$t5], $f38=[$t6], $f39=[$t7], $f40=[$t8], $f41=[$t9], $f42=[$t10], $f43=[$t11], $f44=[$t12], $f45=[$t13], $f46=[$t14], $f47=[$t15], $f48=[$t16], $f49=[$t17], $f50=[$t18], $f51=[$t19], $f52=[$t0], $f53=[$t1], $f54=[$t2], $f55=[$t3], $f56=[$t4])
+  EnumerableJoin(condition=[=($6, $20)], joinType=[inner])
+    EnumerableJoin(condition=[=($0, $5)], joinType=[inner])
+      EnumerableCalc(expr#0..4=[{inputs}], expr#5=['Snacks'], expr#6=[=($t3, $t5)], proj#0..4=[{exprs}], $condition=[$t6])
+        EnumerableTableScan(table=[[foodmart2, product_class]])
+      EnumerableTableScan(table=[[foodmart2, product]])
+    EnumerableJoin(condition=[=($2, $8)], joinType=[inner])
+      EnumerableTableScan(table=[[foodmart2, sales_fact_1997]])
+      EnumerableCalc(expr#0..28=[{inputs}], expr#29=['San Francisco'], expr#30=[=($t9, $t29)], proj#0..28=[{exprs}], $condition=[$t30])
+        EnumerableTableScan(table=[[foodmart2, customer]])
 !plan
 
 # Check that when filters are merged, duplicate conditions are eliminated.
@@ -468,8 +468,8 @@ select * from (
   select * from "days"
   where "day" = 1)
 where "day" = 1;
-EnumerableCalcRel(expr#0..1=[{inputs}], expr#2=[1], expr#3=[=($t0, $t2)], proj#0..1=[{exprs}], $condition=[$t3])
-  EnumerableTableAccessRel(table=[[foodmart2, days]])
+EnumerableCalc(expr#0..1=[{inputs}], expr#2=[1], expr#3=[=($t0, $t2)], proj#0..1=[{exprs}], $condition=[$t3])
+  EnumerableTableScan(table=[[foodmart2, days]])
 !plan
 
 # [HIVE-5873] Semi-join to count subquery

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/resources/sql/subquery.oq
----------------------------------------------------------------------
diff --git a/core/src/test/resources/sql/subquery.oq b/core/src/test/resources/sql/subquery.oq
index fd29c0e..409ca4e 100644
--- a/core/src/test/resources/sql/subquery.oq
+++ b/core/src/test/resources/sql/subquery.oq
@@ -32,31 +32,31 @@ where t1.x not in (select t2.x from t2);
 (0 rows)
 
 !ok
-EnumerableCalcRel(expr#0..5=[{inputs}], expr#6=[0], expr#7=[=($t1, $t6)], expr#8=[false], expr#9=[IS NOT NULL($t5)], expr#10=[true], expr#11=[IS NULL($t3)], expr#12=[null], expr#13=[<($t2, $t1)], expr#14=[CASE($t7, $t8, $t9, $t10, $t11, $t12, $t13, $t12, $t8)], expr#15=[NOT($t14)], X=[$t0], $condition=[$t15])
-  EnumerableJoinRel(condition=[=($3, $4)], joinType=[left])
-    EnumerableCalcRel(expr#0..2=[{inputs}], $f0=[$t2], $f1=[$t0], $f2=[$t1], $f3=[$t2])
-      EnumerableJoinRel(condition=[true], joinType=[inner])
-        EnumerableAggregateRel(group=[{}], agg#0=[COUNT()], agg#1=[COUNT($0)])
-          EnumerableCalcRel(expr#0=[{inputs}], expr#1=[true], proj#0..1=[{exprs}])
-            EnumerableUnionRel(all=[true])
-              EnumerableCalcRel(expr#0=[{inputs}], expr#1=[1], EXPR$0=[$t1])
-                EnumerableValuesRel(tuples=[[{ 0 }]])
-              EnumerableCalcRel(expr#0=[{inputs}], expr#1=[1], expr#2=[=($t1, $t1)], expr#3=[null], expr#4=[3], expr#5=[CASE($t2, $t3, $t4)], EXPR$0=[$t5])
-                EnumerableValuesRel(tuples=[[{ 0 }]])
-        EnumerableUnionRel(all=[true])
-          EnumerableCalcRel(expr#0=[{inputs}], expr#1=[1], EXPR$0=[$t1])
-            EnumerableValuesRel(tuples=[[{ 0 }]])
-          EnumerableCalcRel(expr#0=[{inputs}], expr#1=[2], EXPR$0=[$t1])
-            EnumerableValuesRel(tuples=[[{ 0 }]])
-          EnumerableCalcRel(expr#0=[{inputs}], expr#1=[1], expr#2=[=($t1, $t1)], expr#3=[null], expr#4=[3], expr#5=[CASE($t2, $t3, $t4)], EXPR$0=[$t5])
-            EnumerableValuesRel(tuples=[[{ 0 }]])
-    EnumerableAggregateRel(group=[{0}], agg#0=[MIN($1)])
-      EnumerableCalcRel(expr#0=[{inputs}], expr#1=[true], proj#0..1=[{exprs}])
-        EnumerableUnionRel(all=[true])
-          EnumerableCalcRel(expr#0=[{inputs}], expr#1=[1], EXPR$0=[$t1])
-            EnumerableValuesRel(tuples=[[{ 0 }]])
-          EnumerableCalcRel(expr#0=[{inputs}], expr#1=[1], expr#2=[=($t1, $t1)], expr#3=[null], expr#4=[3], expr#5=[CASE($t2, $t3, $t4)], EXPR$0=[$t5])
-            EnumerableValuesRel(tuples=[[{ 0 }]])
+EnumerableCalc(expr#0..5=[{inputs}], expr#6=[0], expr#7=[=($t1, $t6)], expr#8=[false], expr#9=[IS NOT NULL($t5)], expr#10=[true], expr#11=[IS NULL($t3)], expr#12=[null], expr#13=[<($t2, $t1)], expr#14=[CASE($t7, $t8, $t9, $t10, $t11, $t12, $t13, $t12, $t8)], expr#15=[NOT($t14)], X=[$t0], $condition=[$t15])
+  EnumerableJoin(condition=[=($3, $4)], joinType=[left])
+    EnumerableCalc(expr#0..2=[{inputs}], $f0=[$t2], $f1=[$t0], $f2=[$t1], $f3=[$t2])
+      EnumerableJoin(condition=[true], joinType=[inner])
+        EnumerableAggregate(group=[{}], agg#0=[COUNT()], agg#1=[COUNT($0)])
+          EnumerableCalc(expr#0=[{inputs}], expr#1=[true], proj#0..1=[{exprs}])
+            EnumerableUnion(all=[true])
+              EnumerableCalc(expr#0=[{inputs}], expr#1=[1], EXPR$0=[$t1])
+                EnumerableValues(tuples=[[{ 0 }]])
+              EnumerableCalc(expr#0=[{inputs}], expr#1=[1], expr#2=[=($t1, $t1)], expr#3=[null], expr#4=[3], expr#5=[CASE($t2, $t3, $t4)], EXPR$0=[$t5])
+                EnumerableValues(tuples=[[{ 0 }]])
+        EnumerableUnion(all=[true])
+          EnumerableCalc(expr#0=[{inputs}], expr#1=[1], EXPR$0=[$t1])
+            EnumerableValues(tuples=[[{ 0 }]])
+          EnumerableCalc(expr#0=[{inputs}], expr#1=[2], EXPR$0=[$t1])
+            EnumerableValues(tuples=[[{ 0 }]])
+          EnumerableCalc(expr#0=[{inputs}], expr#1=[1], expr#2=[=($t1, $t1)], expr#3=[null], expr#4=[3], expr#5=[CASE($t2, $t3, $t4)], EXPR$0=[$t5])
+            EnumerableValues(tuples=[[{ 0 }]])
+    EnumerableAggregate(group=[{0}], agg#0=[MIN($1)])
+      EnumerableCalc(expr#0=[{inputs}], expr#1=[true], proj#0..1=[{exprs}])
+        EnumerableUnion(all=[true])
+          EnumerableCalc(expr#0=[{inputs}], expr#1=[1], EXPR$0=[$t1])
+            EnumerableValues(tuples=[[{ 0 }]])
+          EnumerableCalc(expr#0=[{inputs}], expr#1=[1], expr#2=[=($t1, $t1)], expr#3=[null], expr#4=[3], expr#5=[CASE($t2, $t3, $t4)], EXPR$0=[$t5])
+            EnumerableValues(tuples=[[{ 0 }]])
 !plan
 
 # Use of case is to get around issue with directly specifying null in values

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvEnumerator.java
----------------------------------------------------------------------
diff --git a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvEnumerator.java b/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvEnumerator.java
index 7879774..99d788a 100644
--- a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvEnumerator.java
+++ b/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvEnumerator.java
@@ -14,20 +14,23 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl.csv;
+package org.apache.calcite.adapter.csv;
 
-import net.hydromatic.linq4j.Enumerator;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.linq4j.Enumerator;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.util.Pair;
 
-import net.hydromatic.optiq.impl.java.JavaTypeFactory;
-
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.util.Pair;
+import org.apache.commons.lang3.time.FastDateFormat;
 
 import au.com.bytecode.opencsv.CSVReader;
 
-import org.apache.commons.lang3.time.FastDateFormat;
-
-import java.io.*;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileReader;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.Reader;
 import java.text.ParseException;
 import java.util.ArrayList;
 import java.util.Date;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvFieldType.java
----------------------------------------------------------------------
diff --git a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvFieldType.java b/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvFieldType.java
index ad2cd0c..7810926 100644
--- a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvFieldType.java
+++ b/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvFieldType.java
@@ -14,13 +14,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl.csv;
+package org.apache.calcite.adapter.csv;
 
-import net.hydromatic.linq4j.expressions.Primitive;
-
-import net.hydromatic.optiq.impl.java.JavaTypeFactory;
-
-import org.eigenbase.reltype.RelDataType;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.linq4j.tree.Primitive;
+import org.apache.calcite.rel.type.RelDataType;
 
 import java.util.HashMap;
 import java.util.Map;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvFilterableTable.java
----------------------------------------------------------------------
diff --git a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvFilterableTable.java b/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvFilterableTable.java
index 1af0bb7..4ca26f7 100644
--- a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvFilterableTable.java
+++ b/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvFilterableTable.java
@@ -14,15 +14,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl.csv;
+package org.apache.calcite.adapter.csv;
 
-import net.hydromatic.linq4j.*;
-
-import net.hydromatic.optiq.*;
-
-import org.eigenbase.reltype.RelProtoDataType;
-import org.eigenbase.rex.*;
-import org.eigenbase.sql.SqlKind;
+import org.apache.calcite.DataContext;
+import org.apache.calcite.linq4j.AbstractEnumerable;
+import org.apache.calcite.linq4j.Enumerable;
+import org.apache.calcite.linq4j.Enumerator;
+import org.apache.calcite.rel.type.RelProtoDataType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.schema.FilterableTable;
+import org.apache.calcite.sql.SqlKind;
 
 import java.io.File;
 import java.util.Iterator;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvProjectTableScanRule.java
----------------------------------------------------------------------
diff --git a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvProjectTableScanRule.java b/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvProjectTableScanRule.java
index 03e2f17..b0851c7 100644
--- a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvProjectTableScanRule.java
+++ b/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvProjectTableScanRule.java
@@ -14,13 +14,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl.csv;
+package org.apache.calcite.adapter.csv;
 
-import org.eigenbase.rel.ProjectRel;
-import org.eigenbase.relopt.RelOptRule;
-import org.eigenbase.relopt.RelOptRuleCall;
-import org.eigenbase.rex.RexInputRef;
-import org.eigenbase.rex.RexNode;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.rel.logical.LogicalProject;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexNode;
 
 import java.util.List;
 
@@ -29,20 +29,19 @@ import java.util.List;
  * needed to satisfy a projection. If the projection's expressions are trivial,
  * the projection is removed.
  */
-public class CsvPushProjectOntoTableRule extends RelOptRule {
-  public static final CsvPushProjectOntoTableRule INSTANCE =
-      new CsvPushProjectOntoTableRule();
+public class CsvProjectTableScanRule extends RelOptRule {
+  public static final CsvProjectTableScanRule INSTANCE =
+      new CsvProjectTableScanRule();
 
-  private CsvPushProjectOntoTableRule() {
+  private CsvProjectTableScanRule() {
     super(
-        operand(ProjectRel.class,
+        operand(LogicalProject.class,
             operand(CsvTableScan.class, none())),
-        "CsvPushProjectOntoTableRule");
+        "CsvProjectTableScanRule");
   }
 
-  @Override
-  public void onMatch(RelOptRuleCall call) {
-    final ProjectRel project = call.rel(0);
+  @Override public void onMatch(RelOptRuleCall call) {
+    final LogicalProject project = call.rel(0);
     final CsvTableScan scan = call.rel(1);
     int[] fields = getProjectFields(project.getProjects());
     if (fields == null) {
@@ -71,4 +70,4 @@ public class CsvPushProjectOntoTableRule extends RelOptRule {
   }
 }
 
-// End CsvPushProjectOntoTableRule.java
+// End CsvProjectTableScanRule.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvScannableTable.java
----------------------------------------------------------------------
diff --git a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvScannableTable.java b/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvScannableTable.java
index 660270d..005d15b 100644
--- a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvScannableTable.java
+++ b/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvScannableTable.java
@@ -14,14 +14,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl.csv;
-
-import net.hydromatic.linq4j.*;
-
-import net.hydromatic.optiq.DataContext;
-import net.hydromatic.optiq.ScannableTable;
-
-import org.eigenbase.reltype.*;
+package org.apache.calcite.adapter.csv;
+
+import org.apache.calcite.DataContext;
+import org.apache.calcite.linq4j.AbstractEnumerable;
+import org.apache.calcite.linq4j.Enumerable;
+import org.apache.calcite.linq4j.Enumerator;
+import org.apache.calcite.rel.type.RelProtoDataType;
+import org.apache.calcite.schema.ScannableTable;
 
 import java.io.File;
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvSchema.java
----------------------------------------------------------------------
diff --git a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvSchema.java b/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvSchema.java
index 7570828..f4e6a5d 100644
--- a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvSchema.java
+++ b/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvSchema.java
@@ -14,15 +14,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl.csv;
+package org.apache.calcite.adapter.csv;
 
-import net.hydromatic.optiq.*;
-import net.hydromatic.optiq.impl.AbstractSchema;
+import org.apache.calcite.schema.Table;
+import org.apache.calcite.schema.impl.AbstractSchema;
 
 import com.google.common.collect.ImmutableMap;
 
-import java.io.*;
-import java.util.*;
+import java.io.File;
+import java.io.FilenameFilter;
+import java.util.Map;
 
 /**
  * Schema mapped onto a directory of CSV files. Each table in the schema
@@ -62,8 +63,7 @@ public class CsvSchema extends AbstractSchema {
         : null;
   }
 
-  @Override
-  protected Map<String, Table> getTableMap() {
+  @Override protected Map<String, Table> getTableMap() {
     // Look for files in the directory ending in ".csv", ".csv.gz", ".json",
     // ".json.gz".
     File[] files = directoryFile.listFiles(

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvSchemaFactory.java
----------------------------------------------------------------------
diff --git a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvSchemaFactory.java b/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvSchemaFactory.java
index f499cdf..9f3c2bd 100644
--- a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvSchemaFactory.java
+++ b/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvSchemaFactory.java
@@ -14,10 +14,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl.csv;
+package org.apache.calcite.adapter.csv;
 
-import net.hydromatic.optiq.*;
-import net.hydromatic.optiq.model.ModelHandler;
+import org.apache.calcite.model.ModelHandler;
+import org.apache.calcite.schema.Schema;
+import org.apache.calcite.schema.SchemaFactory;
+import org.apache.calcite.schema.SchemaPlus;
 
 import java.io.File;
 import java.util.Map;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvTable.java
----------------------------------------------------------------------
diff --git a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvTable.java b/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvTable.java
index 1d87c0b..05faf18 100644
--- a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvTable.java
+++ b/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvTable.java
@@ -14,12 +14,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl.csv;
+package org.apache.calcite.adapter.csv;
 
-import net.hydromatic.optiq.impl.AbstractTable;
-import net.hydromatic.optiq.impl.java.JavaTypeFactory;
-
-import org.eigenbase.reltype.*;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelProtoDataType;
+import org.apache.calcite.schema.impl.AbstractTable;
 
 import java.io.File;
 import java.util.ArrayList;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvTableFactory.java
----------------------------------------------------------------------
diff --git a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvTableFactory.java b/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvTableFactory.java
index aef815f..9c279b9 100644
--- a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvTableFactory.java
+++ b/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvTableFactory.java
@@ -14,12 +14,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl.csv;
+package org.apache.calcite.adapter.csv;
 
-import net.hydromatic.optiq.*;
-import net.hydromatic.optiq.model.ModelHandler;
-
-import org.eigenbase.reltype.*;
+import org.apache.calcite.model.ModelHandler;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeImpl;
+import org.apache.calcite.rel.type.RelProtoDataType;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.schema.TableFactory;
 
 import java.io.File;
 import java.util.Map;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvTableScan.java
----------------------------------------------------------------------
diff --git a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvTableScan.java b/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvTableScan.java
index e260d5c..8a53e84 100644
--- a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvTableScan.java
+++ b/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvTableScan.java
@@ -14,24 +14,35 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl.csv;
+package org.apache.calcite.adapter.csv;
 
-import net.hydromatic.linq4j.expressions.*;
+import org.apache.calcite.adapter.enumerable.EnumerableConvention;
+import org.apache.calcite.adapter.enumerable.EnumerableRel;
+import org.apache.calcite.adapter.enumerable.EnumerableRelImplementor;
+import org.apache.calcite.adapter.enumerable.PhysType;
+import org.apache.calcite.adapter.enumerable.PhysTypeImpl;
+import org.apache.calcite.linq4j.tree.Blocks;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.linq4j.tree.Primitive;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelWriter;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeField;
 
-import net.hydromatic.optiq.rules.java.*;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.*;
-
-import java.util.*;
+import java.util.List;
 
 /**
  * Relational expression representing a scan of a CSV file.
  *
  * <p>Like any table scan, it serves as a leaf node of a query tree.</p>
  */
-public class CsvTableScan extends TableAccessRelBase implements EnumerableRel {
+public class CsvTableScan extends TableScan implements EnumerableRel {
   final CsvTranslatableTable csvTable;
   final int[] fields;
 
@@ -44,20 +55,17 @@ public class CsvTableScan extends TableAccessRelBase implements EnumerableRel {
     assert csvTable != null;
   }
 
-  @Override
-  public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+  @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
     assert inputs.isEmpty();
     return new CsvTableScan(getCluster(), table, csvTable, fields);
   }
 
-  @Override
-  public RelWriter explainTerms(RelWriter pw) {
+  @Override public RelWriter explainTerms(RelWriter pw) {
     return super.explainTerms(pw)
         .item("fields", Primitive.asList(fields));
   }
 
-  @Override
-  public RelDataType deriveRowType() {
+  @Override public RelDataType deriveRowType() {
     final List<RelDataTypeField> fieldList = table.getRowType().getFieldList();
     final RelDataTypeFactory.FieldInfoBuilder builder =
         getCluster().getTypeFactory().builder();
@@ -67,9 +75,8 @@ public class CsvTableScan extends TableAccessRelBase implements EnumerableRel {
     return builder.build();
   }
 
-  @Override
-  public void register(RelOptPlanner planner) {
-    planner.addRule(CsvPushProjectOntoTableRule.INSTANCE);
+  @Override public void register(RelOptPlanner planner) {
+    planner.addRule(CsvProjectTableScanRule.INSTANCE);
   }
 
   public Result implement(EnumerableRelImplementor implementor, Prefer pref) {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvTranslatableTable.java
----------------------------------------------------------------------
diff --git a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvTranslatableTable.java b/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvTranslatableTable.java
index 3318187..5ae9880 100644
--- a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvTranslatableTable.java
+++ b/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvTranslatableTable.java
@@ -14,23 +14,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl.csv;
+package org.apache.calcite.adapter.csv;
 
-import net.hydromatic.linq4j.AbstractEnumerable;
-import net.hydromatic.linq4j.Enumerable;
-import net.hydromatic.linq4j.Enumerator;
-import net.hydromatic.linq4j.QueryProvider;
-import net.hydromatic.linq4j.Queryable;
-import net.hydromatic.linq4j.expressions.Expression;
-
-import net.hydromatic.optiq.QueryableTable;
-import net.hydromatic.optiq.SchemaPlus;
-import net.hydromatic.optiq.Schemas;
-import net.hydromatic.optiq.TranslatableTable;
-
-import org.eigenbase.rel.RelNode;
-import org.eigenbase.relopt.RelOptTable;
-import org.eigenbase.reltype.RelProtoDataType;
+import org.apache.calcite.linq4j.AbstractEnumerable;
+import org.apache.calcite.linq4j.Enumerable;
+import org.apache.calcite.linq4j.Enumerator;
+import org.apache.calcite.linq4j.QueryProvider;
+import org.apache.calcite.linq4j.Queryable;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.type.RelProtoDataType;
+import org.apache.calcite.schema.QueryableTable;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.schema.Schemas;
+import org.apache.calcite.schema.TranslatableTable;
 
 import java.io.File;
 import java.lang.reflect.Type;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/example/csv/src/main/java/org/apache/calcite/adapter/csv/JsonEnumerator.java
----------------------------------------------------------------------
diff --git a/example/csv/src/main/java/org/apache/calcite/adapter/csv/JsonEnumerator.java b/example/csv/src/main/java/org/apache/calcite/adapter/csv/JsonEnumerator.java
index 2a7c7e4..e129798 100644
--- a/example/csv/src/main/java/org/apache/calcite/adapter/csv/JsonEnumerator.java
+++ b/example/csv/src/main/java/org/apache/calcite/adapter/csv/JsonEnumerator.java
@@ -14,15 +14,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl.csv;
+package org.apache.calcite.adapter.csv;
 
-import net.hydromatic.linq4j.Enumerator;
-import net.hydromatic.linq4j.Linq4j;
+import org.apache.calcite.linq4j.Enumerator;
+import org.apache.calcite.linq4j.Linq4j;
 
 import com.fasterxml.jackson.core.JsonParser;
 import com.fasterxml.jackson.databind.ObjectMapper;
 
-import java.io.*;
+import java.io.File;
+import java.io.IOException;
 import java.util.List;
 
 /** Enumerator that reads from a JSON file. */

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/example/csv/src/main/java/org/apache/calcite/adapter/csv/JsonTable.java
----------------------------------------------------------------------
diff --git a/example/csv/src/main/java/org/apache/calcite/adapter/csv/JsonTable.java b/example/csv/src/main/java/org/apache/calcite/adapter/csv/JsonTable.java
index 6e3c10a..95b0247 100644
--- a/example/csv/src/main/java/org/apache/calcite/adapter/csv/JsonTable.java
+++ b/example/csv/src/main/java/org/apache/calcite/adapter/csv/JsonTable.java
@@ -14,17 +14,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl.csv;
-
-import net.hydromatic.linq4j.*;
-
-import net.hydromatic.optiq.*;
-import net.hydromatic.optiq.impl.AbstractTable;
-
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.type.SqlTypeName;
-
-import java.io.*;
+package org.apache.calcite.adapter.csv;
+
+import org.apache.calcite.DataContext;
+import org.apache.calcite.linq4j.AbstractEnumerable;
+import org.apache.calcite.linq4j.Enumerable;
+import org.apache.calcite.linq4j.Enumerator;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.schema.ScannableTable;
+import org.apache.calcite.schema.impl.AbstractTable;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+import java.io.File;
 
 /**
  * Table based on a JSON file.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/example/csv/src/main/java/org/apache/calcite/adapter/csv/package-info.java
----------------------------------------------------------------------
diff --git a/example/csv/src/main/java/org/apache/calcite/adapter/csv/package-info.java b/example/csv/src/main/java/org/apache/calcite/adapter/csv/package-info.java
index f0a10e7..f4b2d6e 100644
--- a/example/csv/src/main/java/org/apache/calcite/adapter/csv/package-info.java
+++ b/example/csv/src/main/java/org/apache/calcite/adapter/csv/package-info.java
@@ -22,6 +22,6 @@
  * directory appears as a table.  Full SQL operations are available on
  * those tables.</p>
  */
-package net.hydromatic.optiq.impl.csv;
+package org.apache.calcite.adapter.csv;
 
 // End package-info.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/example/csv/src/test/java/org/apache/calcite/test/CsvTest.java
----------------------------------------------------------------------
diff --git a/example/csv/src/test/java/org/apache/calcite/test/CsvTest.java b/example/csv/src/test/java/org/apache/calcite/test/CsvTest.java
index 7803c2a..85187af 100644
--- a/example/csv/src/test/java/org/apache/calcite/test/CsvTest.java
+++ b/example/csv/src/test/java/org/apache/calcite/test/CsvTest.java
@@ -14,9 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.test;
+package org.apache.calcite.test;
 
-import net.hydromatic.linq4j.function.Function1;
+import org.apache.calcite.linq4j.function.Function1;
 
 import org.junit.Assert;
 import org.junit.Ignore;
@@ -24,14 +24,19 @@ import org.junit.Test;
 
 import java.io.PrintStream;
 import java.net.URL;
-import java.sql.*;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+import java.sql.Statement;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 import java.util.Properties;
 
 /**
- * Unit test of the Optiq adapter for CSV.
+ * Unit test of the Calcite adapter for CSV.
  */
 public class CsvTest {
   private void close(Connection connection, Statement statement) {
@@ -72,8 +77,7 @@ public class CsvTest {
   @Ignore
   @Test public void testVanityDriverArgsInUrl() throws SQLException {
     Connection connection =
-        DriverManager.getConnection(
-            "jdbc:csv:"
+        DriverManager.getConnection("jdbc:csv:"
             + "directory='foo'");
     connection.close();
   }
@@ -83,19 +87,19 @@ public class CsvTest {
     Properties info = new Properties();
     info.put("model",
         "inline:"
-        + "{\n"
-        + "  version: '1.0',\n"
-        + "   schemas: [\n"
-        + "     {\n"
-        + "       type: 'custom',\n"
-        + "       name: 'bad',\n"
-        + "       factory: 'net.hydromatic.optiq.impl.csv.CsvSchemaFactory',\n"
-        + "       operand: {\n"
-        + "         directory: '/does/not/exist'\n"
-        + "       }\n"
-        + "     }\n"
-        + "   ]\n"
-        + "}");
+            + "{\n"
+            + "  version: '1.0',\n"
+            + "   schemas: [\n"
+            + "     {\n"
+            + "       type: 'custom',\n"
+            + "       name: 'bad',\n"
+            + "       factory: 'org.apache.calcite.adapter.csv.CsvSchemaFactory',\n"
+            + "       operand: {\n"
+            + "         directory: '/does/not/exist'\n"
+            + "       }\n"
+            + "     }\n"
+            + "   ]\n"
+            + "}");
 
     Connection connection =
         DriverManager.getConnection("jdbc:calcite:", info);
@@ -129,7 +133,7 @@ public class CsvTest {
   @Test public void testPushDownProjectDumb() throws SQLException {
     // rule does not fire, because we're using 'dumb' tables in simple model
     checkSql("model", "explain plan for select * from EMPS",
-        "PLAN=EnumerableTableAccessRel(table=[[SALES, EMPS]])\n");
+        "PLAN=EnumerableTableScan(table=[[SALES, EMPS]])\n");
   }
 
   @Test public void testPushDownProject() throws SQLException {
@@ -173,9 +177,9 @@ public class CsvTest {
 
   @Test public void testJson() throws SQLException {
     checkSql("bug", "select _MAP['id'] as id,\n"
-        + " _MAP['title'] as title,\n"
-        + " CHAR_LENGTH(CAST(_MAP['title'] AS VARCHAR(30))) as len\n"
-        + " from \"archers\"",
+            + " _MAP['title'] as title,\n"
+            + " CHAR_LENGTH(CAST(_MAP['title'] AS VARCHAR(30))) as len\n"
+            + " from \"archers\"",
         "ID=19990101; TITLE=Washday blues.; LEN=14",
         "ID=19990103; TITLE=Daniel creates a drama.; LEN=23");
   }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/example/csv/src/test/java/org/apache/calcite/test/package-info.java
----------------------------------------------------------------------
diff --git a/example/csv/src/test/java/org/apache/calcite/test/package-info.java b/example/csv/src/test/java/org/apache/calcite/test/package-info.java
deleted file mode 100644
index a42c1f6..0000000
--- a/example/csv/src/test/java/org/apache/calcite/test/package-info.java
+++ /dev/null
@@ -1,23 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to you under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-/**
- * Unit tests.
- */
-package net.hydromatic.optiq.test;
-
-// End package-info.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/example/csv/src/test/resources/bug.json
----------------------------------------------------------------------
diff --git a/example/csv/src/test/resources/bug.json b/example/csv/src/test/resources/bug.json
index 8f9222b..1b28466 100644
--- a/example/csv/src/test/resources/bug.json
+++ b/example/csv/src/test/resources/bug.json
@@ -5,7 +5,7 @@
     {
       name: 'BUG',
       type: 'custom',
-      factory: 'net.hydromatic.optiq.impl.csv.CsvSchemaFactory',
+      factory: 'org.apache.calcite.adapter.csv.CsvSchemaFactory',
       operand: {
         directory: 'bug'
       }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/example/csv/src/test/resources/filterable-model.json
----------------------------------------------------------------------
diff --git a/example/csv/src/test/resources/filterable-model.json b/example/csv/src/test/resources/filterable-model.json
index 74fb046..f632d16 100644
--- a/example/csv/src/test/resources/filterable-model.json
+++ b/example/csv/src/test/resources/filterable-model.json
@@ -8,7 +8,7 @@
     {
       name: 'SALES',
       type: 'custom',
-      factory: 'net.hydromatic.optiq.impl.csv.CsvSchemaFactory',
+      factory: 'org.apache.calcite.adapter.csv.CsvSchemaFactory',
       operand: {
         directory: 'sales',
         flavor: "FILTERABLE"

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/example/csv/src/test/resources/model-with-custom-table.json
----------------------------------------------------------------------
diff --git a/example/csv/src/test/resources/model-with-custom-table.json b/example/csv/src/test/resources/model-with-custom-table.json
index ab68c00..ccc7615 100644
--- a/example/csv/src/test/resources/model-with-custom-table.json
+++ b/example/csv/src/test/resources/model-with-custom-table.json
@@ -8,7 +8,7 @@
         {
           name: 'EMPS',
           type: 'custom',
-          factory: 'net.hydromatic.optiq.impl.csv.CsvTableFactory',
+          factory: 'org.apache.calcite.adapter.csv.CsvTableFactory',
           operand: {
             file: 'sales/EMPS.csv.gz',
             flavor: "scannable"

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/example/csv/src/test/resources/model-with-view.json
----------------------------------------------------------------------
diff --git a/example/csv/src/test/resources/model-with-view.json b/example/csv/src/test/resources/model-with-view.json
index 8174a3f..dae0a2d 100644
--- a/example/csv/src/test/resources/model-with-view.json
+++ b/example/csv/src/test/resources/model-with-view.json
@@ -5,7 +5,7 @@
     {
       name: 'SALES',
       type: 'custom',
-      factory: 'net.hydromatic.optiq.impl.csv.CsvSchemaFactory',
+      factory: 'org.apache.calcite.adapter.csv.CsvSchemaFactory',
       operand: {
         directory: 'target/test-classes/sales'
       },

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/example/csv/src/test/resources/model.json
----------------------------------------------------------------------
diff --git a/example/csv/src/test/resources/model.json b/example/csv/src/test/resources/model.json
index 7e241eb..50b1b95 100644
--- a/example/csv/src/test/resources/model.json
+++ b/example/csv/src/test/resources/model.json
@@ -5,7 +5,7 @@
     {
       name: 'SALES',
       type: 'custom',
-      factory: 'net.hydromatic.optiq.impl.csv.CsvSchemaFactory',
+      factory: 'org.apache.calcite.adapter.csv.CsvSchemaFactory',
       operand: {
         directory: 'sales'
       }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/example/csv/src/test/resources/smart.json
----------------------------------------------------------------------
diff --git a/example/csv/src/test/resources/smart.json b/example/csv/src/test/resources/smart.json
index bebc2d8..d6ec5bf 100644
--- a/example/csv/src/test/resources/smart.json
+++ b/example/csv/src/test/resources/smart.json
@@ -9,7 +9,7 @@
     {
       name: 'SALES',
       type: 'custom',
-      factory: 'net.hydromatic.optiq.impl.csv.CsvSchemaFactory',
+      factory: 'org.apache.calcite.adapter.csv.CsvSchemaFactory',
       operand: {
         directory: 'sales',
         flavor: "TRANSLATABLE"

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/pom.xml
----------------------------------------------------------------------
diff --git a/linq4j/pom.xml b/linq4j/pom.xml
index 6fd3f9e..a9aca65 100644
--- a/linq4j/pom.xml
+++ b/linq4j/pom.xml
@@ -43,7 +43,7 @@
         <version>2.15</version>
         <configuration>
           <includes>
-            <include>net/hydromatic/linq4j/test/Linq4jSuite.java</include>
+            <include>org/apache/calcite/linq4j/test/Linq4jSuite.java</include>
           </includes>
         </configuration>
       </plugin>

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/AbstractEnumerable.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/AbstractEnumerable.java b/linq4j/src/main/java/org/apache/calcite/linq4j/AbstractEnumerable.java
index 2bd2c89..ef3f648 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/AbstractEnumerable.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/AbstractEnumerable.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j;
+package org.apache.calcite.linq4j;
 
 import java.util.Iterator;
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/AbstractEnumerable2.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/AbstractEnumerable2.java b/linq4j/src/main/java/org/apache/calcite/linq4j/AbstractEnumerable2.java
index fc0989d..b6fbbb9 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/AbstractEnumerable2.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/AbstractEnumerable2.java
@@ -14,11 +14,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j;
+package org.apache.calcite.linq4j;
 
 /**
- * Abstract implementation of the {@link net.hydromatic.linq4j.Enumerable} interface that
- * implements the extension methods.
+ * Abstract implementation of the {@link org.apache.calcite.linq4j.Enumerable}
+ * interface that implements the extension methods.
  *
  * <p>It is helpful to derive from this class if you are implementing
  * {@code Enumerable}, because {@code Enumerable} has so many extension methods,

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/AbstractQueryable.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/AbstractQueryable.java b/linq4j/src/main/java/org/apache/calcite/linq4j/AbstractQueryable.java
index c2e00d8..958b119 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/AbstractQueryable.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/AbstractQueryable.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j;
+package org.apache.calcite.linq4j;
 
 /**
  * Abstract implementation of the {@link Queryable} interface that implements

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/BaseQueryable.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/BaseQueryable.java b/linq4j/src/main/java/org/apache/calcite/linq4j/BaseQueryable.java
index 2120e1f..9014712 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/BaseQueryable.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/BaseQueryable.java
@@ -14,9 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j;
+package org.apache.calcite.linq4j;
 
-import net.hydromatic.linq4j.expressions.Expression;
+import org.apache.calcite.linq4j.tree.Expression;
 
 import java.lang.reflect.Type;
 import java.util.Iterator;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/CartesianProductEnumerator.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/CartesianProductEnumerator.java b/linq4j/src/main/java/org/apache/calcite/linq4j/CartesianProductEnumerator.java
index 84c1c7b..88fbd5c 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/CartesianProductEnumerator.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/CartesianProductEnumerator.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j;
+package org.apache.calcite.linq4j;
 
 import java.util.Arrays;
 import java.util.List;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/DefaultEnumerable.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/DefaultEnumerable.java b/linq4j/src/main/java/org/apache/calcite/linq4j/DefaultEnumerable.java
index 490874d..a0853bf 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/DefaultEnumerable.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/DefaultEnumerable.java
@@ -14,16 +14,34 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j;
-
-import net.hydromatic.linq4j.function.*;
+package org.apache.calcite.linq4j;
+
+import org.apache.calcite.linq4j.function.BigDecimalFunction1;
+import org.apache.calcite.linq4j.function.DoubleFunction1;
+import org.apache.calcite.linq4j.function.EqualityComparer;
+import org.apache.calcite.linq4j.function.FloatFunction1;
+import org.apache.calcite.linq4j.function.Function0;
+import org.apache.calcite.linq4j.function.Function1;
+import org.apache.calcite.linq4j.function.Function2;
+import org.apache.calcite.linq4j.function.IntegerFunction1;
+import org.apache.calcite.linq4j.function.LongFunction1;
+import org.apache.calcite.linq4j.function.NullableBigDecimalFunction1;
+import org.apache.calcite.linq4j.function.NullableDoubleFunction1;
+import org.apache.calcite.linq4j.function.NullableFloatFunction1;
+import org.apache.calcite.linq4j.function.NullableIntegerFunction1;
+import org.apache.calcite.linq4j.function.NullableLongFunction1;
+import org.apache.calcite.linq4j.function.Predicate1;
+import org.apache.calcite.linq4j.function.Predicate2;
 
 import java.math.BigDecimal;
-import java.util.*;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
 
 /**
- * Implementation of the {@link net.hydromatic.linq4j.Enumerable} interface that
- * implements the extension methods by calling into the {@link Extensions}
+ * Implementation of the {@link org.apache.calcite.linq4j.Enumerable} interface
+ * that implements the extension methods by calling into the {@link Extensions}
  * class.
  *
  * <p>The are two abstract methods:

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/DefaultQueryable.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/DefaultQueryable.java b/linq4j/src/main/java/org/apache/calcite/linq4j/DefaultQueryable.java
index c762aec..5c7468b 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/DefaultQueryable.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/DefaultQueryable.java
@@ -14,10 +14,24 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j;
-
-import net.hydromatic.linq4j.expressions.FunctionExpression;
-import net.hydromatic.linq4j.function.*;
+package org.apache.calcite.linq4j;
+
+import org.apache.calcite.linq4j.function.BigDecimalFunction1;
+import org.apache.calcite.linq4j.function.DoubleFunction1;
+import org.apache.calcite.linq4j.function.EqualityComparer;
+import org.apache.calcite.linq4j.function.FloatFunction1;
+import org.apache.calcite.linq4j.function.Function1;
+import org.apache.calcite.linq4j.function.Function2;
+import org.apache.calcite.linq4j.function.IntegerFunction1;
+import org.apache.calcite.linq4j.function.LongFunction1;
+import org.apache.calcite.linq4j.function.NullableBigDecimalFunction1;
+import org.apache.calcite.linq4j.function.NullableDoubleFunction1;
+import org.apache.calcite.linq4j.function.NullableFloatFunction1;
+import org.apache.calcite.linq4j.function.NullableIntegerFunction1;
+import org.apache.calcite.linq4j.function.NullableLongFunction1;
+import org.apache.calcite.linq4j.function.Predicate1;
+import org.apache.calcite.linq4j.function.Predicate2;
+import org.apache.calcite.linq4j.tree.FunctionExpression;
 
 import java.math.BigDecimal;
 import java.util.Comparator;
@@ -48,8 +62,7 @@ abstract class DefaultQueryable<T> extends DefaultEnumerable<T>
   }
 
   // override return type
-  @Override
-  protected Queryable<T> getThis() {
+  @Override protected Queryable<T> getThis() {
     return this;
   }
 
@@ -57,8 +70,7 @@ abstract class DefaultQueryable<T> extends DefaultEnumerable<T>
     return this;
   }
 
-  @Override
-  public Enumerable<T> asEnumerable() {
+  @Override public Enumerable<T> asEnumerable() {
     return new AbstractEnumerable<T>() {
       public Enumerator<T> enumerator() {
         return DefaultQueryable.this.enumerator();
@@ -68,81 +80,66 @@ abstract class DefaultQueryable<T> extends DefaultEnumerable<T>
 
   // Disambiguate
 
-  @Override
-  public Queryable<T> union(Enumerable<T> source1) {
+  @Override public Queryable<T> union(Enumerable<T> source1) {
     return factory.union(getThis(), source1);
   }
 
-  @Override
-  public Queryable<T> union(Enumerable<T> source1,
+  @Override public Queryable<T> union(Enumerable<T> source1,
       EqualityComparer<T> comparer) {
     return factory.union(getThis(), source1, comparer);
   }
 
-  @Override
-  public Queryable<T> intersect(Enumerable<T> source1) {
+  @Override public Queryable<T> intersect(Enumerable<T> source1) {
     return factory.intersect(getThis(), source1);
   }
 
-  @Override
-  public Queryable<T> intersect(Enumerable<T> source1,
+  @Override public Queryable<T> intersect(Enumerable<T> source1,
       EqualityComparer<T> comparer) {
     return factory.intersect(getThis(), source1, comparer);
   }
 
-  @Override
-  public Queryable<T> except(Enumerable<T> enumerable1,
+  @Override public Queryable<T> except(Enumerable<T> enumerable1,
       EqualityComparer<T> comparer) {
     return factory.except(getThis(), enumerable1, comparer);
   }
 
-  @Override
-  public Queryable<T> except(Enumerable<T> enumerable1) {
+  @Override public Queryable<T> except(Enumerable<T> enumerable1) {
     return factory.except(getThis(), enumerable1);
   }
 
-  @Override
-  public Queryable<T> take(int count) {
+  @Override public Queryable<T> take(int count) {
     return factory.take(getThis(), count);
   }
 
-  @Override
-  public Queryable<T> skip(int count) {
+  @Override public Queryable<T> skip(int count) {
     return factory.skip(getThis(), count);
   }
 
-  @Override
-  public Queryable<T> reverse() {
+  @Override public Queryable<T> reverse() {
     return factory.reverse(getThis());
   }
 
-  @Override
-  public Queryable<T> distinct() {
+  @Override public Queryable<T> distinct() {
     return factory.distinct(getThis());
   }
 
-  @Override
-  public Queryable<T> distinct(EqualityComparer<T> comparer) {
+  @Override public Queryable<T> distinct(EqualityComparer<T> comparer) {
     return factory.distinct(getThis(), comparer);
   }
 
-  @Override
-  public <TResult> Queryable<TResult> ofType(Class<TResult> clazz) {
+  @Override public <TResult> Queryable<TResult> ofType(Class<TResult> clazz) {
     return factory.ofType(getThis(), clazz);
   }
 
-  @Override
-  public Queryable<T> defaultIfEmpty() {
+  @Override public Queryable<T> defaultIfEmpty() {
     return factory.defaultIfEmpty(getThis());
   }
 
-  @Override
-  public Queryable<T> asQueryable() {
+  @Override public Queryable<T> asQueryable() {
     return this;
   }
 
-  @Override
-  public <T2> Queryable<T2> cast(Class<T2> clazz) {
+  @Override public <T2> Queryable<T2> cast(Class<T2> clazz) {
     return factory.cast(getThis(), clazz);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/Enumerable.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/Enumerable.java b/linq4j/src/main/java/org/apache/calcite/linq4j/Enumerable.java
index c2a1680..060b974 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/Enumerable.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/Enumerable.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j;
+package org.apache.calcite.linq4j;
 
 /**
  * Exposes the enumerator, which supports a simple iteration over a collection.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java b/linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
index f1b889d..0e2d436 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
@@ -14,14 +14,47 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j;
-
-import net.hydromatic.linq4j.function.*;
+package org.apache.calcite.linq4j;
+
+import org.apache.calcite.linq4j.function.BigDecimalFunction1;
+import org.apache.calcite.linq4j.function.DoubleFunction1;
+import org.apache.calcite.linq4j.function.EqualityComparer;
+import org.apache.calcite.linq4j.function.FloatFunction1;
+import org.apache.calcite.linq4j.function.Function0;
+import org.apache.calcite.linq4j.function.Function1;
+import org.apache.calcite.linq4j.function.Function2;
+import org.apache.calcite.linq4j.function.Functions;
+import org.apache.calcite.linq4j.function.IntegerFunction1;
+import org.apache.calcite.linq4j.function.LongFunction1;
+import org.apache.calcite.linq4j.function.NullableBigDecimalFunction1;
+import org.apache.calcite.linq4j.function.NullableDoubleFunction1;
+import org.apache.calcite.linq4j.function.NullableFloatFunction1;
+import org.apache.calcite.linq4j.function.NullableIntegerFunction1;
+import org.apache.calcite.linq4j.function.NullableLongFunction1;
+import org.apache.calcite.linq4j.function.Predicate1;
+import org.apache.calcite.linq4j.function.Predicate2;
 
 import java.math.BigDecimal;
-import java.util.*;
-
-import static net.hydromatic.linq4j.function.Functions.adapt;
+import java.util.AbstractList;
+import java.util.AbstractMap;
+import java.util.AbstractSet;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.RandomAccess;
+import java.util.Set;
+import java.util.TreeMap;
+
+import static org.apache.calcite.linq4j.function.Functions.adapt;
 
 /**
  * Default implementations of methods in the {@link Enumerable} interface.
@@ -2250,13 +2283,11 @@ public abstract class EnumerableDefaults {
       return new Wrapped<T>(comparer, element);
     }
 
-    @Override
-    public int hashCode() {
+    @Override public int hashCode() {
       return comparer.hashCode(element);
     }
 
-    @Override
-    public boolean equals(Object obj) {
+    @Override public boolean equals(Object obj) {
       //noinspection unchecked
       return obj == this || obj instanceof Wrapped && comparer.equal(element,
           ((Wrapped<T>) obj).element);
@@ -2276,11 +2307,9 @@ public abstract class EnumerableDefaults {
       this.comparer = comparer;
     }
 
-    @Override
-    public Set<Entry<K, V>> entrySet() {
+    @Override public Set<Entry<K, V>> entrySet() {
       return new AbstractSet<Entry<K, V>>() {
-        @Override
-        public Iterator<Entry<K, V>> iterator() {
+        @Override public Iterator<Entry<K, V>> iterator() {
           final Iterator<Entry<Wrapped<K>, V>> iterator =
               map.entrySet().iterator();
 
@@ -2301,15 +2330,13 @@ public abstract class EnumerableDefaults {
           };
         }
 
-        @Override
-        public int size() {
+        @Override public int size() {
           return map.size();
         }
       };
     }
 
-    @Override
-    public boolean containsKey(Object key) {
+    @Override public boolean containsKey(Object key) {
       return map.containsKey(wrap((K) key));
     }
 
@@ -2317,28 +2344,23 @@ public abstract class EnumerableDefaults {
       return Wrapped.upAs(comparer, key);
     }
 
-    @Override
-    public V get(Object key) {
+    @Override public V get(Object key) {
       return map.get(wrap((K) key));
     }
 
-    @Override
-    public V put(K key, V value) {
+    @Override public V put(K key, V value) {
       return map.put(wrap(key), value);
     }
 
-    @Override
-    public V remove(Object key) {
+    @Override public V remove(Object key) {
       return map.remove(wrap((K) key));
     }
 
-    @Override
-    public void clear() {
+    @Override public void clear() {
       map.clear();
     }
 
-    @Override
-    public Collection<V> values() {
+    @Override public Collection<V> values() {
       return map.values();
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableOrderedQueryable.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableOrderedQueryable.java b/linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableOrderedQueryable.java
index 69df280..dc610b1 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableOrderedQueryable.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableOrderedQueryable.java
@@ -14,17 +14,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j;
+package org.apache.calcite.linq4j;
 
-import net.hydromatic.linq4j.expressions.Expression;
-import net.hydromatic.linq4j.expressions.FunctionExpression;
-import net.hydromatic.linq4j.function.*;
+import org.apache.calcite.linq4j.function.Function1;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.FunctionExpression;
 
 import java.util.Comparator;
 
 /**
  * Implementation of {@link OrderedQueryable} by an
- * {@link net.hydromatic.linq4j.Enumerable}.
+ * {@link org.apache.calcite.linq4j.Enumerable}.
  *
  * @param <T> Element type
  */

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableQueryable.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableQueryable.java b/linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableQueryable.java
index 02a3610..4e55bc8 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableQueryable.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableQueryable.java
@@ -14,11 +14,25 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j;
-
-import net.hydromatic.linq4j.expressions.Expression;
-import net.hydromatic.linq4j.expressions.FunctionExpression;
-import net.hydromatic.linq4j.function.*;
+package org.apache.calcite.linq4j;
+
+import org.apache.calcite.linq4j.function.BigDecimalFunction1;
+import org.apache.calcite.linq4j.function.DoubleFunction1;
+import org.apache.calcite.linq4j.function.EqualityComparer;
+import org.apache.calcite.linq4j.function.FloatFunction1;
+import org.apache.calcite.linq4j.function.Function1;
+import org.apache.calcite.linq4j.function.Function2;
+import org.apache.calcite.linq4j.function.IntegerFunction1;
+import org.apache.calcite.linq4j.function.LongFunction1;
+import org.apache.calcite.linq4j.function.NullableBigDecimalFunction1;
+import org.apache.calcite.linq4j.function.NullableDoubleFunction1;
+import org.apache.calcite.linq4j.function.NullableFloatFunction1;
+import org.apache.calcite.linq4j.function.NullableIntegerFunction1;
+import org.apache.calcite.linq4j.function.NullableLongFunction1;
+import org.apache.calcite.linq4j.function.Predicate1;
+import org.apache.calcite.linq4j.function.Predicate2;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.FunctionExpression;
 
 import java.lang.reflect.Type;
 import java.math.BigDecimal;
@@ -45,8 +59,7 @@ class EnumerableQueryable<T> extends DefaultEnumerable<T>
     this.expression = expression;
   }
 
-  @Override
-  protected Enumerable<T> getThis() {
+  @Override protected Enumerable<T> getThis() {
     return enumerable;
   }
 
@@ -78,27 +91,23 @@ class EnumerableQueryable<T> extends DefaultEnumerable<T>
     return EnumerableDefaults.union(getThis(), source1, comparer).asQueryable();
   }
 
-  @Override
-  public Queryable<T> intersect(Enumerable<T> source1) {
+  @Override public Queryable<T> intersect(Enumerable<T> source1) {
     return EnumerableDefaults.intersect(getThis(), source1).asQueryable();
   }
 
-  @Override
-  public Queryable<T> intersect(Enumerable<T> source1,
+  @Override public Queryable<T> intersect(Enumerable<T> source1,
       EqualityComparer<T> comparer) {
     return EnumerableDefaults.intersect(getThis(), source1, comparer)
         .asQueryable();
   }
 
-  @Override
-  public Queryable<T> except(Enumerable<T> enumerable1,
+  @Override public Queryable<T> except(Enumerable<T> enumerable1,
       EqualityComparer<T> comparer) {
     return EnumerableDefaults.except(getThis(), enumerable1, comparer)
         .asQueryable();
   }
 
-  @Override
-  public Queryable<T> except(Enumerable<T> enumerable1) {
+  @Override public Queryable<T> except(Enumerable<T> enumerable1) {
     return EnumerableDefaults.except(getThis(), enumerable1).asQueryable();
   }
 
@@ -114,23 +123,19 @@ class EnumerableQueryable<T> extends DefaultEnumerable<T>
     return EnumerableDefaults.reverse(getThis()).asQueryable();
   }
 
-  @Override
-  public Queryable<T> distinct() {
+  @Override public Queryable<T> distinct() {
     return EnumerableDefaults.distinct(getThis()).asQueryable();
   }
 
-  @Override
-  public Queryable<T> distinct(EqualityComparer<T> comparer) {
+  @Override public Queryable<T> distinct(EqualityComparer<T> comparer) {
     return EnumerableDefaults.distinct(getThis(), comparer).asQueryable();
   }
 
-  @Override
-  public <TResult> Queryable<TResult> ofType(Class<TResult> clazz) {
+  @Override public <TResult> Queryable<TResult> ofType(Class<TResult> clazz) {
     return EnumerableDefaults.ofType(getThis(), clazz).asQueryable();
   }
 
-  @Override
-  public Queryable<T> defaultIfEmpty() {
+  @Override public Queryable<T> defaultIfEmpty() {
     return EnumerableDefaults.defaultIfEmpty(getThis()).asQueryable();
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/Enumerator.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/Enumerator.java b/linq4j/src/main/java/org/apache/calcite/linq4j/Enumerator.java
index f308456..143e644 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/Enumerator.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/Enumerator.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j;
+package org.apache.calcite.linq4j;
 
 import java.io.Closeable;
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/ExtendedEnumerable.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/ExtendedEnumerable.java b/linq4j/src/main/java/org/apache/calcite/linq4j/ExtendedEnumerable.java
index fed663c..f657dd7 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/ExtendedEnumerable.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/ExtendedEnumerable.java
@@ -14,12 +14,30 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j;
-
-import net.hydromatic.linq4j.function.*;
+package org.apache.calcite.linq4j;
+
+import org.apache.calcite.linq4j.function.BigDecimalFunction1;
+import org.apache.calcite.linq4j.function.DoubleFunction1;
+import org.apache.calcite.linq4j.function.EqualityComparer;
+import org.apache.calcite.linq4j.function.FloatFunction1;
+import org.apache.calcite.linq4j.function.Function0;
+import org.apache.calcite.linq4j.function.Function1;
+import org.apache.calcite.linq4j.function.Function2;
+import org.apache.calcite.linq4j.function.IntegerFunction1;
+import org.apache.calcite.linq4j.function.LongFunction1;
+import org.apache.calcite.linq4j.function.NullableBigDecimalFunction1;
+import org.apache.calcite.linq4j.function.NullableDoubleFunction1;
+import org.apache.calcite.linq4j.function.NullableFloatFunction1;
+import org.apache.calcite.linq4j.function.NullableIntegerFunction1;
+import org.apache.calcite.linq4j.function.NullableLongFunction1;
+import org.apache.calcite.linq4j.function.Predicate1;
+import org.apache.calcite.linq4j.function.Predicate2;
 
 import java.math.BigDecimal;
-import java.util.*;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
 
 /**
  * Extension methods in {@link Enumerable}.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/ExtendedOrderedEnumerable.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/ExtendedOrderedEnumerable.java b/linq4j/src/main/java/org/apache/calcite/linq4j/ExtendedOrderedEnumerable.java
index c34ae94..d8f81cb 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/ExtendedOrderedEnumerable.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/ExtendedOrderedEnumerable.java
@@ -14,9 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j;
+package org.apache.calcite.linq4j;
 
-import net.hydromatic.linq4j.function.Function1;
+import org.apache.calcite.linq4j.function.Function1;
 
 import java.util.Comparator;
 
@@ -32,8 +32,9 @@ public interface ExtendedOrderedEnumerable<T> extends Enumerable<T> {
    * comparator.
    *
    * <p>The functionality provided by this method is like that provided by
-   * {@link #thenBy(net.hydromatic.linq4j.function.Function1, java.util.Comparator) thenBy}
-   * or {@link #thenByDescending(net.hydromatic.linq4j.function.Function1, java.util.Comparator) thenByDescending},
+   * {@link #thenBy(org.apache.calcite.linq4j.function.Function1, java.util.Comparator) thenBy}
+   * or
+   * {@link #thenByDescending(org.apache.calcite.linq4j.function.Function1, java.util.Comparator) thenByDescending},
    * depending on whether descending is true or false. They both perform a
    * subordinate ordering of an already sorted sequence of type
    * {@link OrderedEnumerable}.</p>

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/ExtendedOrderedQueryable.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/ExtendedOrderedQueryable.java b/linq4j/src/main/java/org/apache/calcite/linq4j/ExtendedOrderedQueryable.java
index a4aec1e..bdeaddc 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/ExtendedOrderedQueryable.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/ExtendedOrderedQueryable.java
@@ -14,10 +14,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j;
+package org.apache.calcite.linq4j;
 
-import net.hydromatic.linq4j.expressions.FunctionExpression;
-import net.hydromatic.linq4j.function.Function1;
+import org.apache.calcite.linq4j.function.Function1;
+import org.apache.calcite.linq4j.tree.FunctionExpression;
 
 import java.util.Comparator;
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/ExtendedQueryable.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/ExtendedQueryable.java b/linq4j/src/main/java/org/apache/calcite/linq4j/ExtendedQueryable.java
index 091f208..bbefb7c 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/ExtendedQueryable.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/ExtendedQueryable.java
@@ -14,10 +14,24 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j;
-
-import net.hydromatic.linq4j.expressions.FunctionExpression;
-import net.hydromatic.linq4j.function.*;
+package org.apache.calcite.linq4j;
+
+import org.apache.calcite.linq4j.function.BigDecimalFunction1;
+import org.apache.calcite.linq4j.function.DoubleFunction1;
+import org.apache.calcite.linq4j.function.EqualityComparer;
+import org.apache.calcite.linq4j.function.FloatFunction1;
+import org.apache.calcite.linq4j.function.Function1;
+import org.apache.calcite.linq4j.function.Function2;
+import org.apache.calcite.linq4j.function.IntegerFunction1;
+import org.apache.calcite.linq4j.function.LongFunction1;
+import org.apache.calcite.linq4j.function.NullableBigDecimalFunction1;
+import org.apache.calcite.linq4j.function.NullableDoubleFunction1;
+import org.apache.calcite.linq4j.function.NullableFloatFunction1;
+import org.apache.calcite.linq4j.function.NullableIntegerFunction1;
+import org.apache.calcite.linq4j.function.NullableLongFunction1;
+import org.apache.calcite.linq4j.function.Predicate1;
+import org.apache.calcite.linq4j.function.Predicate2;
+import org.apache.calcite.linq4j.tree.FunctionExpression;
 
 import java.math.BigDecimal;
 import java.util.Comparator;
@@ -231,7 +245,7 @@ interface ExtendedQueryable<TSource> extends ExtendedEnumerable<TSource> {
    * each group and its key.
    *
    * <p>NOTE: Renamed from {@code groupBy} to distinguish from
-   * {@link #groupBy(net.hydromatic.linq4j.expressions.FunctionExpression, net.hydromatic.linq4j.expressions.FunctionExpression)},
+   * {@link #groupBy(org.apache.calcite.linq4j.tree.FunctionExpression, org.apache.calcite.linq4j.tree.FunctionExpression)},
    * which has the same erasure.</p>
    */
   <TKey, TResult> Queryable<Grouping<TKey, TResult>> groupByK(
@@ -386,7 +400,7 @@ interface ExtendedQueryable<TSource> extends ExtendedEnumerable<TSource> {
    * specified type.
    *
    * <p>The OfType method generates a
-   * {@link net.hydromatic.linq4j.expressions.MethodCallExpression} that represents
+   * {@link org.apache.calcite.linq4j.tree.MethodCallExpression} that represents
    * calling OfType itself as a constructed generic method. It then passes the
    * MethodCallExpression to the CreateQuery(Expression) method of the
    * {@link QueryProvider} represented by the Provider property of the source
@@ -451,7 +465,7 @@ interface ExtendedQueryable<TSource> extends ExtendedEnumerable<TSource> {
    * form by incorporating the element's index.
    *
    * <p>NOTE: Renamed from {@code select} because had same erasure as
-   * {@link #select(net.hydromatic.linq4j.expressions.FunctionExpression)}.</p>
+   * {@link #select(org.apache.calcite.linq4j.tree.FunctionExpression)}.</p>
    */
   <TResult> Queryable<TResult> selectN(
       FunctionExpression<Function2<TSource, Integer, TResult>> selector);
@@ -472,7 +486,7 @@ interface ExtendedQueryable<TSource> extends ExtendedEnumerable<TSource> {
    * projected form of that element.
    *
    * <p>NOTE: Renamed from {@code selectMany} because had same erasure as
-   * {@link #selectMany(net.hydromatic.linq4j.expressions.FunctionExpression)}</p>
+   * {@link #selectMany(org.apache.calcite.linq4j.tree.FunctionExpression)}</p>
    */
   <TResult> Queryable<TResult> selectManyN(
       FunctionExpression<Function2<TSource, Integer, Enumerable<TResult>>>
@@ -500,7 +514,7 @@ interface ExtendedQueryable<TSource> extends ExtendedEnumerable<TSource> {
    * and returned.
    *
    * <p>NOTE: Renamed from {@code selectMany} because had same erasure as
-   * {@link #selectMany(net.hydromatic.linq4j.expressions.FunctionExpression, net.hydromatic.linq4j.expressions.FunctionExpression)}</p>
+   * {@link #selectMany(org.apache.calcite.linq4j.tree.FunctionExpression, org.apache.calcite.linq4j.tree.FunctionExpression)}
    */
   <TCollection, TResult> Queryable<TResult> selectManyN(
       FunctionExpression<Function1<TSource, Enumerable<TCollection>>>

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/Extensions.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/Extensions.java b/linq4j/src/main/java/org/apache/calcite/linq4j/Extensions.java
index c66a2d4..c72e88c 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/Extensions.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/Extensions.java
@@ -14,12 +14,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j;
+package org.apache.calcite.linq4j;
 
-import net.hydromatic.linq4j.function.*;
+import org.apache.calcite.linq4j.function.Function2;
 
 import java.math.BigDecimal;
-import java.util.*;
+import java.util.Comparator;
+import java.util.Map;
 
 /**
  * Contains what, in LINQ.NET, would be extension methods.
@@ -35,7 +36,7 @@ import java.util.*;
  *
  * <li>A few methods became keywords when their first letter was converted
  * to lower case; hence
- * {@link net.hydromatic.linq4j.expressions.Expressions#break_}</li>
+ * {@link org.apache.calcite.linq4j.tree.Expressions#break_}</li>
  *
  * <li>We created a Java interface {@link Enumerable}, similar to LINQ.NET's
  * IEnumerable. IEnumerable is built into C#, and that gives it
@@ -55,7 +56,8 @@ import java.util.*;
  *
  * <li>There is no Nullable in Java. Therefore we distinguish between methods
  * that return, say, Long (which may be null) and long. See for example
- * {@link NullableLongFunction1} and {@link LongFunction1}, and the
+ * {@link org.apache.calcite.linq4j.function.NullableLongFunction1} and
+ * {@link org.apache.calcite.linq4j.function.LongFunction1}, and the
  * variants of {@link Enumerable#sum} that call them.
  *
  * <li>Java erases type parameters from argument types before resolving
@@ -84,9 +86,9 @@ import java.util.*;
  * in the interface. Hence {@link AbstractEnumerable} and
  * {@link AbstractQueryable} call methods in {@link Extensions}.</li>
  *
- * <li>.NET Func becomes {@link net.hydromatic.linq4j.function.Function0},
- * {@link net.hydromatic.linq4j.function.Function1},
- * {@link net.hydromatic.linq4j.function.Function2}, depending
+ * <li>.NET Func becomes {@link org.apache.calcite.linq4j.function.Function0},
+ * {@link org.apache.calcite.linq4j.function.Function1},
+ * {@link org.apache.calcite.linq4j.function.Function2}, depending
  * on the number of arguments to the function, because Java types cannot be
  * overloaded based on the number of type parameters.</li>
  *


[11/58] [abbrv] [partial] incubator-calcite git commit: [CALCITE-306] Standardize code style for "import package.*; "

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
index df99443..3f90177 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
@@ -14,33 +14,36 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.test;
-
-import java.nio.charset.*;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Locale;
-import java.util.logging.*;
-
-import org.eigenbase.reltype.RelDataTypeSystem;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.fun.SqlStdOperatorTable;
-import org.eigenbase.sql.test.SqlTester;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.sql.validate.*;
-import org.eigenbase.util.*;
-
-import net.hydromatic.avatica.Casing;
-import net.hydromatic.avatica.Quoting;
-
-import net.hydromatic.optiq.config.Lex;
+package org.apache.calcite.test;
+
+import org.apache.calcite.avatica.Casing;
+import org.apache.calcite.avatica.Quoting;
+import org.apache.calcite.config.Lex;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.sql.SqlCollation;
+import org.apache.calcite.sql.SqlIntervalQualifier;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.test.SqlTester;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.validate.SqlConformance;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.util.Bug;
 
 import org.hamcrest.CoreMatchers;
 import org.junit.BeforeClass;
 import org.junit.Ignore;
 import org.junit.Test;
 
-import static org.junit.Assert.*;
+import java.nio.charset.Charset;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Locale;
+import java.util.logging.Logger;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
 
 /**
  * Concrete child class of {@link SqlValidatorTestCase}, containing lots of unit
@@ -48,7 +51,7 @@ import static org.junit.Assert.*;
  *
  * <p>If you want to run these same tests in a different environment, create a
  * derived class whose {@link #getTester} returns a different implementation of
- * {@link org.eigenbase.sql.test.SqlTester}.
+ * {@link org.apache.calcite.sql.test.SqlTester}.
  */
 public class SqlValidatorTest extends SqlValidatorTestCase {
   //~ Static fields/initializers ---------------------------------------------
@@ -81,14 +84,14 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
 
   private static final String EMP_RECORD_TYPE =
       "RecordType(INTEGER NOT NULL EMPNO,"
-      + " VARCHAR(20) NOT NULL ENAME,"
-      + " VARCHAR(10) NOT NULL JOB,"
-      + " INTEGER MGR,"
-      + " TIMESTAMP(0) NOT NULL HIREDATE,"
-      + " INTEGER NOT NULL SAL,"
-      + " INTEGER NOT NULL COMM,"
-      + " INTEGER NOT NULL DEPTNO,"
-      + " BOOLEAN NOT NULL SLACKER) NOT NULL";
+          + " VARCHAR(20) NOT NULL ENAME,"
+          + " VARCHAR(10) NOT NULL JOB,"
+          + " INTEGER MGR,"
+          + " TIMESTAMP(0) NOT NULL HIREDATE,"
+          + " INTEGER NOT NULL SAL,"
+          + " INTEGER NOT NULL COMM,"
+          + " INTEGER NOT NULL DEPTNO,"
+          + " BOOLEAN NOT NULL SLACKER) NOT NULL";
 
   //~ Constructors -----------------------------------------------------------
 
@@ -105,8 +108,7 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
   }
 
   @Test public void testMultipleSameAsPass() {
-    check(
-        "select 1 as again,2 as \"again\", 3 as AGAiN from (values (true))");
+    check("select 1 as again,2 as \"again\", 3 as AGAiN from (values (true))");
   }
 
   @Test public void testMultipleDifferentAs() {
@@ -136,17 +138,17 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
         + "' cd'", "CHAR(6) NOT NULL");
     checkExpType(
         "'ab'\n"
-        + "'cd'\n"
-        + "'ef'\n"
-        + "'gh'\n"
-        + "'ij'\n"
-        + "'kl'",
+            + "'cd'\n"
+            + "'ef'\n"
+            + "'gh'\n"
+            + "'ij'\n"
+            + "'kl'",
         "CHAR(12) NOT NULL");
     checkExpType("n'ab '\n"
-        + "' cd'",
+            + "' cd'",
         "CHAR(6) NOT NULL");
     checkExpType("_UTF16'ab '\n"
-        + "' cd'",
+            + "' cd'",
         "CHAR(6) NOT NULL");
 
     checkExpFails(
@@ -154,13 +156,13 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
         "Binary literal string must contain an even number of hexits");
     checkExpType("x'abcd'", "BINARY(2) NOT NULL");
     checkExpType("x'abcd'\n"
-        + "'ff001122aabb'",
+            + "'ff001122aabb'",
         "BINARY(8) NOT NULL");
     checkExpType(
         "x'aaaa'\n"
-        + "'bbbb'\n"
-        + "'0000'\n"
-        + "'1111'",
+            + "'bbbb'\n"
+            + "'0000'\n"
+            + "'1111'",
         "BINARY(8) NOT NULL");
 
     checkExpType("1234567890", "INTEGER NOT NULL");
@@ -516,9 +518,9 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
         "(?s).*ELSE clause or at least one THEN clause must be non-NULL.*");
     checkWholeExpFails(
         "case when true and true then 1 "
-        + "when false then 2 "
-        + "when false then true " + "else "
-        + "case when true then 3 end end",
+            + "when false then 2 "
+            + "when false then true " + "else "
+            + "case when true then 3 end end",
         "Illegal mixing of types in CASE or COALESCE statement");
   }
 
@@ -599,7 +601,7 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     checkWholeExpFails(
         "'a'||x'ff'",
         "(?s).*Cannot apply '\\|\\|' to arguments of type '<CHAR.1.> \\|\\| <BINARY.1.>'"
-        + ".*Supported form.s.: '<STRING> \\|\\| <STRING>.*'");
+            + ".*Supported form.s.: '<STRING> \\|\\| <STRING>.*'");
   }
 
   @Test public void testBetween() {
@@ -1170,7 +1172,7 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
   @Test public void testRow() {
     // double-nested rows can confuse validator namespace resolution
     checkColumnType("select t.r.\"EXPR$1\".\"EXPR$2\"\n"
-        + "from (select ((1,2),(3,4,5)) r from dept) t",
+            + "from (select ((1,2),(3,4,5)) r from dept) t",
         "INTEGER NOT NULL");
   }
 
@@ -1202,14 +1204,14 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
         "RecordType(DECIMAL(11, 1) NOT NULL EXPR$0, DECIMAL(11, 1) NOT NULL EXPR$1) NOT NULL MULTISET NOT NULL");
     checkExpType("multiset(select*from emp)",
         "RecordType(INTEGER NOT NULL EMPNO,"
-        + " VARCHAR(20) NOT NULL ENAME,"
-        + " VARCHAR(10) NOT NULL JOB,"
-        + " INTEGER MGR,"
-        + " TIMESTAMP(0) NOT NULL HIREDATE,"
-        + " INTEGER NOT NULL SAL,"
-        + " INTEGER NOT NULL COMM,"
-        + " INTEGER NOT NULL DEPTNO,"
-        + " BOOLEAN NOT NULL SLACKER) NOT NULL MULTISET NOT NULL");
+            + " VARCHAR(20) NOT NULL ENAME,"
+            + " VARCHAR(10) NOT NULL JOB,"
+            + " INTEGER MGR,"
+            + " TIMESTAMP(0) NOT NULL HIREDATE,"
+            + " INTEGER NOT NULL SAL,"
+            + " INTEGER NOT NULL COMM,"
+            + " INTEGER NOT NULL DEPTNO,"
+            + " BOOLEAN NOT NULL SLACKER) NOT NULL MULTISET NOT NULL");
   }
 
   @Test public void testMultisetSetOperators() {
@@ -1275,8 +1277,8 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     checkWholeExpFails(
         "cardinality('a')",
         "Cannot apply 'CARDINALITY' to arguments of type 'CARDINALITY\\(<CHAR\\(1\\)>\\)'\\. Supported form\\(s\\): 'CARDINALITY\\(<MULTISET>\\)'\n"
-        + "'CARDINALITY\\(<ARRAY>\\)'\n"
-        + "'CARDINALITY\\(<MAP>\\)'");
+            + "'CARDINALITY\\(<ARRAY>\\)'\n"
+            + "'CARDINALITY\\(<MAP>\\)'");
   }
 
   @Test public void testIntervalTimeUnitEnumeration() {
@@ -3271,25 +3273,25 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     checkWholeExpFails(
         "INTERVAL '1:60' MINUTE TO SECOND",
         "Illegal interval literal format '1:60' for"
-        + " INTERVAL MINUTE TO SECOND.*");
+            + " INTERVAL MINUTE TO SECOND.*");
     checkWholeExpFails(
         "INTERVAL '1:1.0000001' MINUTE TO SECOND",
         "Illegal interval literal format '1:1\\.0000001' for"
-        + " INTERVAL MINUTE TO SECOND.*");
+            + " INTERVAL MINUTE TO SECOND.*");
     checkWholeExpFails(
         "INTERVAL '1:1:1.0001' MINUTE TO SECOND(3)",
         "Illegal interval literal format '1:1:1\\.0001' for"
-        + " INTERVAL MINUTE TO SECOND\\(3\\).*");
+            + " INTERVAL MINUTE TO SECOND\\(3\\).*");
 
     // precision > maximum
     checkExpFails(
         "INTERVAL '1:1' MINUTE(11) TO ^SECOND^",
         "Interval leading field precision '11' out of range for"
-        + " INTERVAL MINUTE\\(11\\) TO SECOND");
+            + " INTERVAL MINUTE\\(11\\) TO SECOND");
     checkExpFails(
         "INTERVAL '1:1' MINUTE TO SECOND(10^)^",
         "Interval fractional second precision '10' out of range for"
-        + " INTERVAL MINUTE TO SECOND\\(10\\)");
+            + " INTERVAL MINUTE TO SECOND\\(10\\)");
 
     // precision < minimum allowed)
     // note: parser will catch negative values, here we
@@ -3297,11 +3299,11 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     checkExpFails(
         "INTERVAL '0:0' MINUTE(0) TO ^SECOND^",
         "Interval leading field precision '0' out of range for"
-        + " INTERVAL MINUTE\\(0\\) TO SECOND");
+            + " INTERVAL MINUTE\\(0\\) TO SECOND");
     checkExpFails(
         "INTERVAL '0:0' MINUTE TO SECOND(0^)^",
         "Interval fractional second precision '0' out of range for"
-        + " INTERVAL MINUTE TO SECOND\\(0\\)");
+            + " INTERVAL MINUTE TO SECOND\\(0\\)");
   }
 
   /**
@@ -3381,17 +3383,17 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     checkWholeExpFails(
         "INTERVAL '1.0000000001' SECOND(2, 9)",
         "Illegal interval literal format '1\\.0000000001' for"
-        + " INTERVAL SECOND\\(2, 9\\).*");
+            + " INTERVAL SECOND\\(2, 9\\).*");
 
     // precision > maximum
     checkExpFails(
         "INTERVAL '1' SECOND(11^)^",
         "Interval leading field precision '11' out of range for"
-        + " INTERVAL SECOND\\(11\\)");
+            + " INTERVAL SECOND\\(11\\)");
     checkExpFails(
         "INTERVAL '1.1' SECOND(1, 10^)^",
         "Interval fractional second precision '10' out of range for"
-        + " INTERVAL SECOND\\(1, 10\\)");
+            + " INTERVAL SECOND\\(1, 10\\)");
 
     // precision < minimum allowed)
     // note: parser will catch negative values, here we
@@ -3399,11 +3401,11 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     checkExpFails(
         "INTERVAL '0' SECOND(0^)^",
         "Interval leading field precision '0' out of range for"
-        + " INTERVAL SECOND\\(0\\)");
+            + " INTERVAL SECOND\\(0\\)");
     checkExpFails(
         "INTERVAL '0' SECOND(1, 0^)^",
         "Interval fractional second precision '0' out of range for"
-        + " INTERVAL SECOND\\(1, 0\\)");
+            + " INTERVAL SECOND\\(1, 0\\)");
   }
 
   @Test public void testIntervalLiterals() {
@@ -3812,29 +3814,29 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     // See 4.15.3 for detail
     checkWin(
         "select *\n"
-        + " from emp\n"
-        + " where ^sum(sal) over (partition by deptno\n"
-        + "    order by empno\n"
-        + "    rows 3 preceding)^ > 10",
+            + " from emp\n"
+            + " where ^sum(sal) over (partition by deptno\n"
+            + "    order by empno\n"
+            + "    rows 3 preceding)^ > 10",
         "Windowed aggregate expression is illegal in WHERE clause");
 
     checkWin(
         "select *\n"
-        + " from emp\n"
-        + " group by ename, ^sum(sal) over (partition by deptno\n"
-        + "    order by empno\n"
-        + "    rows 3 preceding)^ + 10\n"
-        + "order by deptno",
+            + " from emp\n"
+            + " group by ename, ^sum(sal) over (partition by deptno\n"
+            + "    order by empno\n"
+            + "    rows 3 preceding)^ + 10\n"
+            + "order by deptno",
         "Windowed aggregate expression is illegal in GROUP BY clause");
 
     checkWin(
         "select *\n"
-        + " from emp\n"
-        + " join dept on emp.deptno = dept.deptno\n"
-        + " and ^sum(sal) over (partition by deptno\n"
-        + "    order by empno\n"
-        + "    rows 3 preceding)^ = dept.deptno + 40\n"
-        + "order by deptno",
+            + " from emp\n"
+            + " join dept on emp.deptno = dept.deptno\n"
+            + " and ^sum(sal) over (partition by deptno\n"
+            + "    order by empno\n"
+            + "    rows 3 preceding)^ = dept.deptno + 40\n"
+            + "order by deptno",
         "Windowed aggregate expression is illegal in ON clause");
 
     // rule 3, a)
@@ -3858,11 +3860,11 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     }
     checkWin(
         "select rank() over w from emp\n"
-        + "window w as ^(partition by sal)^, w2 as (w order by deptno)",
+            + "window w as ^(partition by sal)^, w2 as (w order by deptno)",
         "RANK or DENSE_RANK functions require ORDER BY clause in window specification");
     checkWin(
         "select rank() over w2 from emp\n"
-        + "window w as (partition by sal), w2 as (w order by deptno)",
+            + "window w as (partition by sal), w2 as (w order by deptno)",
         null);
     // row_number function
     checkWinFuncExpWithWinClause(
@@ -3956,18 +3958,18 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     // 7.11 rule 10c
     checkWin(
         "select sum(sal) over (w partition by ^deptno^)\n"
-        + " from emp window w as (order by empno rows 2 preceding )",
+            + " from emp window w as (order by empno rows 2 preceding )",
         "PARTITION BY not allowed with existing window reference");
 
     // 7.11 rule 10d
     checkWin(
         "select sum(sal) over (w order by ^empno^)\n"
-        + " from emp window w as (order by empno rows 2 preceding )",
+            + " from emp window w as (order by empno rows 2 preceding )",
         "ORDER BY not allowed in both base and referenced windows");
 
     // 7.11 rule 10e
     checkWin("select sum(sal) over (w)\n"
-        + " from emp window w as (order by empno ^rows^ 2 preceding )",
+            + " from emp window w as (order by empno ^rows^ 2 preceding )",
         "Referenced window cannot have framing declarations");
 
     // Empty window is OK for functions that don't require ordering.
@@ -3984,28 +3986,28 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
   @Test public void testInlineWinDef() {
     // the <window specification> used by windowed agg functions is
     // fully defined in SQL 03 Std. section 7.1 <window clause>
-    check(
-        "select sum(sal) over (partition by deptno order by empno) from emp order by empno");
+    check("select sum(sal) over (partition by deptno order by empno)\n"
+        + "from emp order by empno");
     checkWinFuncExp(
         "sum(sal) OVER ("
-        + "partition by deptno "
-        + "order by empno "
-        + "rows 2 preceding )",
+            + "partition by deptno "
+            + "order by empno "
+            + "rows 2 preceding )",
         null);
     checkWinFuncExp(
         "sum(sal) OVER ("
-        + "order by 1 "
-        + "rows 2 preceding )",
+            + "order by 1 "
+            + "rows 2 preceding )",
         null);
     checkWinFuncExp(
         "sum(sal) OVER ("
-        + "order by 'b' "
-        + "rows 2 preceding )",
+            + "order by 'b' "
+            + "rows 2 preceding )",
         null);
     checkWinFuncExp(
         "sum(sal) over ("
-        + "partition by deptno "
-        + "order by 1+1 rows 26 preceding)",
+            + "partition by deptno "
+            + "order by 1+1 rows 26 preceding)",
         null);
     checkWinFuncExp(
         "sum(sal) over (order by deptno rows unbounded preceding)",
@@ -4015,42 +4017,42 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
         null);
     checkWinFuncExp(
         "sum(sal) over ^("
-        + "order by deptno "
-        + "rows between unbounded preceding and unbounded following)^",
+            + "order by deptno "
+            + "rows between unbounded preceding and unbounded following)^",
         null);
     checkWinFuncExp(
         "sum(sal) over ^("
-        + "order by deptno "
-        + "rows between CURRENT ROW and unbounded following)^",
+            + "order by deptno "
+            + "rows between CURRENT ROW and unbounded following)^",
         null);
     checkWinFuncExp(
         "sum(sal) over ("
-        + "order by deptno "
-        + "rows between unbounded preceding and CURRENT ROW)",
+            + "order by deptno "
+            + "rows between unbounded preceding and CURRENT ROW)",
         null);
 
     // logical current row/current row
     checkWinFuncExp(
         "sum(sal) over ("
-        + "order by deptno "
-        + "rows between CURRENT ROW and CURRENT ROW)",
+            + "order by deptno "
+            + "rows between CURRENT ROW and CURRENT ROW)",
         null);
 
     // physical current row/current row
     checkWinFuncExp(
         "sum(sal) over ("
-        + "order by deptno "
-        + "range between CURRENT ROW and CURRENT ROW)",
+            + "order by deptno "
+            + "range between CURRENT ROW and CURRENT ROW)",
         null);
 
     checkWinFuncExp(
         "sum(sal) over ("
-        + "order by deptno "
-        + "rows between 2 preceding and CURRENT ROW)",
+            + "order by deptno "
+            + "rows between 2 preceding and CURRENT ROW)",
         null);
     checkWinFuncExpWithWinClause(
         "sum(sal) OVER (w "
-        + "rows 2 preceding )",
+            + "rows 2 preceding )",
         null);
     checkWinFuncExp(
         "sum(sal) over (order by deptno range 2.0 preceding)",
@@ -4059,37 +4061,37 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     // Failure mode tests
     checkWinFuncExp(
         "sum(sal) over (order by deptno "
-        + "rows between ^UNBOUNDED FOLLOWING^ and unbounded preceding)",
+            + "rows between ^UNBOUNDED FOLLOWING^ and unbounded preceding)",
         "UNBOUNDED FOLLOWING cannot be specified for the lower frame boundary");
     checkWinFuncExp(
         "sum(sal) over ("
-        + "order by deptno "
-        + "rows between 2 preceding and ^UNBOUNDED PRECEDING^)",
+            + "order by deptno "
+            + "rows between 2 preceding and ^UNBOUNDED PRECEDING^)",
         "UNBOUNDED PRECEDING cannot be specified for the upper frame boundary");
     checkWinFuncExp(
         "sum(sal) over ("
-        + "order by deptno "
-        + "rows between CURRENT ROW and ^2 preceding^)",
+            + "order by deptno "
+            + "rows between CURRENT ROW and ^2 preceding^)",
         "Upper frame boundary cannot be PRECEDING when lower boundary is CURRENT ROW");
     checkWinFuncExp(
         "sum(sal) over ("
-        + "order by deptno "
-        + "rows between 2 following and ^CURRENT ROW^)",
+            + "order by deptno "
+            + "rows between 2 following and ^CURRENT ROW^)",
         "Upper frame boundary cannot be CURRENT ROW when lower boundary is FOLLOWING");
     checkWinFuncExp(
         "sum(sal) over ("
-        + "order by deptno "
-        + "rows between 2 following and ^2 preceding^)",
+            + "order by deptno "
+            + "rows between 2 following and ^2 preceding^)",
         "Upper frame boundary cannot be PRECEDING when lower boundary is FOLLOWING");
     checkWinFuncExp(
         "sum(sal) over ("
-        + "order by deptno "
-        + "RANGE BETWEEN ^INTERVAL '1' SECOND^ PRECEDING AND INTERVAL '1' SECOND FOLLOWING)",
+            + "order by deptno "
+            + "RANGE BETWEEN ^INTERVAL '1' SECOND^ PRECEDING AND INTERVAL '1' SECOND FOLLOWING)",
         "Data Type mismatch between ORDER BY and RANGE clause");
     checkWinFuncExp(
         "sum(sal) over ("
-        + "order by empno "
-        + "RANGE BETWEEN ^INTERVAL '1' SECOND^ PRECEDING AND INTERVAL '1' SECOND FOLLOWING)",
+            + "order by empno "
+            + "RANGE BETWEEN ^INTERVAL '1' SECOND^ PRECEDING AND INTERVAL '1' SECOND FOLLOWING)",
         "Data Type mismatch between ORDER BY and RANGE clause");
     checkWinFuncExp(
         "sum(sal) over (order by deptno, empno ^range^ 2 preceding)",
@@ -4102,9 +4104,9 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
         "Window 'W1' not found");
     checkWinFuncExp(
         "sum(sal) OVER (^w1^ "
-        + "partition by deptno "
-        + "order by empno "
-        + "rows 2 preceding )",
+            + "partition by deptno "
+            + "order by empno "
+            + "rows 2 preceding )",
         "Window 'W1' not found");
   }
 
@@ -4211,11 +4213,11 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
         "Expression 'DEPTNO' is not being grouped");
     checkFails(
         "select min(sal) over\n"
-        + "(partition by comm order by deptno) from emp group by deptno,sal,comm",
+            + "(partition by comm order by deptno) from emp group by deptno,sal,comm",
         null);
     checkFails(
         "select min(sal) over\n"
-        + "(partition by ^comm^ order by deptno) from emp group by deptno,sal",
+            + "(partition by ^comm^ order by deptno) from emp group by deptno,sal",
         "Expression 'COMM' is not being grouped");
 
     // syntax rule 7
@@ -4234,23 +4236,23 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     // bound 2 shall not specify UNBOUNDED PRECEDING
     checkWinClauseExp(
         "window w as ("
-        + "order by deptno "
-        + "rows between 2 preceding and ^UNBOUNDED PRECEDING^)",
+            + "order by deptno "
+            + "rows between 2 preceding and ^UNBOUNDED PRECEDING^)",
         "UNBOUNDED PRECEDING cannot be specified for the upper frame boundary");
     checkWinClauseExp(
         "window w as ("
-        + "order by deptno "
-        + "rows between 2 following and ^2 preceding^)",
+            + "order by deptno "
+            + "rows between 2 following and ^2 preceding^)",
         "Upper frame boundary cannot be PRECEDING when lower boundary is FOLLOWING");
     checkWinClauseExp(
         "window w as ("
-        + "order by deptno "
-        + "rows between CURRENT ROW and ^2 preceding^)",
+            + "order by deptno "
+            + "rows between CURRENT ROW and ^2 preceding^)",
         "Upper frame boundary cannot be PRECEDING when lower boundary is CURRENT ROW");
     checkWinClauseExp(
         "window w as ("
-        + "order by deptno "
-        + "rows between 2 following and ^CURRENT ROW^)",
+            + "order by deptno "
+            + "rows between 2 following and ^CURRENT ROW^)",
         "Upper frame boundary cannot be CURRENT ROW when lower boundary is FOLLOWING");
 
     // Sql '03 rule 10 c) assertExceptionIsThrown("select deptno as d, sal
@@ -4296,25 +4298,23 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     // are equivalent.
     checkWinClauseExp(
         "window\n"
-        + "w  as (partition by deptno order by empno rows 2 preceding),\n"
-        + "w2 as ^(partition by deptno order by empno rows 2 preceding)^\n",
+            + "w  as (partition by deptno order by empno rows 2 preceding),\n"
+            + "w2 as ^(partition by deptno order by empno rows 2 preceding)^\n",
         "Duplicate window specification not allowed in the same window clause");
   }
 
   @Test public void testWindowClauseWithSubquery() {
-    check(
-        "select * from\n"
+    check("select * from\n"
         + "( select sum(empno) over w, sum(deptno) over w from emp\n"
         + "window w as (order by hiredate range interval '1' minute preceding))");
 
-    check(
-        "select * from\n"
+    check("select * from\n"
         + "( select sum(empno) over w, sum(deptno) over w, hiredate from emp)\n"
         + "window w as (order by hiredate range interval '1' minute preceding)");
 
     checkFails("select * from\n"
-        + "( select sum(empno) over w, sum(deptno) over w from emp)\n"
-        + "window w as (order by ^hiredate^ range interval '1' minute preceding)",
+            + "( select sum(empno) over w, sum(deptno) over w from emp)\n"
+            + "window w as (order by ^hiredate^ range interval '1' minute preceding)",
         "Column 'HIREDATE' not found in any table");
   }
 
@@ -4348,27 +4348,26 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
   private void checkNegWindow(String s, String msg) {
     String sql =
         "select sum(deptno) over ^(order by empno "
-        + s
-        + ")^ from emp";
+            + s
+            + ")^ from emp";
     checkFails(
         sql,
         msg);
   }
 
   @Test public void testWindowPartial() {
-    check(
-        "select sum(deptno) over (\n"
+    check("select sum(deptno) over (\n"
         + "order by deptno, empno rows 2 preceding disallow partial)\n"
         + "from emp");
 
     // cannot do partial over logical window
     checkFails(
         "select sum(deptno) over (\n"
-        + "  partition by deptno\n"
-        + "  order by empno\n"
-        + "  range between 2 preceding and 3 following\n"
-        + "  ^disallow partial^)\n"
-        + "from emp",
+            + "  partition by deptno\n"
+            + "  order by empno\n"
+            + "  range between 2 preceding and 3 following\n"
+            + "  ^disallow partial^)\n"
+            + "from emp",
         "Cannot use DISALLOW PARTIAL with window based on RANGE");
   }
 
@@ -4386,52 +4385,49 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
 
     checkFails(
         "select * from " + emps + " join " + depts + "\n"
-        + " on ^emps^.deptno = deptno",
+            + " on ^emps^.deptno = deptno",
         "Table 'EMPS' not found");
 
     // this is ok
-    check(
-        "select * from " + emps + " as e\n"
+    check("select * from " + emps + " as e\n"
         + " join " + depts + " as d\n"
         + " on e.deptno = d.deptno");
 
     // fail: ambiguous column in WHERE
     checkFails(
         "select * from " + emps + " as emps,\n"
-        + " " + depts + "\n"
-        + "where ^deptno^ > 5",
+            + " " + depts + "\n"
+            + "where ^deptno^ > 5",
         "Column 'DEPTNO' is ambiguous");
 
     // fail: ambiguous column reference in ON clause
     checkFails(
         "select * from " + emps + " as e\n"
-        + " join " + depts + " as d\n"
-        + " on e.deptno = ^deptno^",
+            + " join " + depts + " as d\n"
+            + " on e.deptno = ^deptno^",
         "Column 'DEPTNO' is ambiguous");
 
     // ok: column 'age' is unambiguous
-    check(
-        "select * from " + emps + " as e\n"
+    check("select * from " + emps + " as e\n"
         + " join " + depts + " as d\n"
         + " on e.deptno = age");
 
     // ok: reference to derived column
-    check(
-        "select * from " + depts + "\n"
+    check("select * from " + depts + "\n"
         + " join (select mod(age, 30) as agemod from " + emps + ")\n"
         + "on deptno = agemod");
 
     // fail: deptno is ambiguous
     checkFails(
         "select name from " + depts + "\n"
-        + "join (select mod(age, 30) as agemod, deptno from " + emps + ")\n"
-        + "on ^deptno^ = agemod",
+            + "join (select mod(age, 30) as agemod, deptno from " + emps + ")\n"
+            + "on ^deptno^ = agemod",
         "Column 'DEPTNO' is ambiguous");
 
     // fail: lateral reference
     checkFails(
         "select * from " + emps + " as e,\n"
-        + " (select 1, ^e^.deptno from (values(true))) as d",
+            + " (select 1, ^e^.deptno from (values(true))) as d",
         "Table 'E' not found");
   }
 
@@ -4446,69 +4442,66 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
         "BOOLEAN NOT NULL");
     checkColumnType(
         "select * from ("
-        + "  select * from ("
-        + "    select * from (values(true))"
-        + "    union"
-        + "    select * from (values (false)))"
-        + "  except"
-        + "  select * from (values(true)))",
+            + "  select * from ("
+            + "    select * from (values(true))"
+            + "    union"
+            + "    select * from (values (false)))"
+            + "  except"
+            + "  select * from (values(true)))",
         "BOOLEAN NOT NULL");
   }
 
   @Test public void testAmbiguousColumn() {
     checkFails(
         "select * from emp join dept\n"
-        + " on emp.deptno = ^deptno^",
+            + " on emp.deptno = ^deptno^",
         "Column 'DEPTNO' is ambiguous");
 
     // this is ok
-    check(
-        "select * from emp as e\n"
+    check("select * from emp as e\n"
         + " join dept as d\n"
         + " on e.deptno = d.deptno");
 
     // fail: ambiguous column in WHERE
     checkFails(
         "select * from emp as emps, dept\n"
-        + "where ^deptno^ > 5",
+            + "where ^deptno^ > 5",
         "Column 'DEPTNO' is ambiguous");
 
     // fail: alias 'd' obscures original table name 'dept'
     checkFails(
         "select * from emp as emps, dept as d\n"
-        + "where ^dept^.deptno > 5",
+            + "where ^dept^.deptno > 5",
         "Table 'DEPT' not found");
 
     // fail: ambiguous column reference in ON clause
     checkFails(
         "select * from emp as e\n"
-        + " join dept as d\n"
-        + " on e.deptno = ^deptno^",
+            + " join dept as d\n"
+            + " on e.deptno = ^deptno^",
         "Column 'DEPTNO' is ambiguous");
 
     // ok: column 'comm' is unambiguous
-    check(
-        "select * from emp as e\n"
+    check("select * from emp as e\n"
         + " join dept as d\n"
         + " on e.deptno = comm");
 
     // ok: reference to derived column
-    check(
-        "select * from dept\n"
+    check("select * from dept\n"
         + " join (select mod(comm, 30) as commmod from emp)\n"
         + "on deptno = commmod");
 
     // fail: deptno is ambiguous
     checkFails(
         "select name from dept\n"
-        + "join (select mod(comm, 30) as commmod, deptno from emp)\n"
-        + "on ^deptno^ = commmod",
+            + "join (select mod(comm, 30) as commmod, deptno from emp)\n"
+            + "on ^deptno^ = commmod",
         "Column 'DEPTNO' is ambiguous");
 
     // fail: lateral reference
     checkFails(
         "select * from emp as e,\n"
-        + " (select 1, ^e^.deptno from (values(true))) as d",
+            + " (select 1, ^e^.deptno from (values(true))) as d",
         "Table 'E' not found");
   }
 
@@ -4558,8 +4551,8 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     check("select * from emp where empno in (10,20)");
 
     // "select * from emp where empno in ()" is invalid -- see parser test
-    check(
-        "select * from emp where empno in (10 + deptno, cast(null as integer))");
+    check("select * from emp\n"
+        + "where empno in (10 + deptno, cast(null as integer))");
     checkFails(
         "select * from emp where empno in ^(10, '20')^",
         ERR_IN_VALUES_INCOMPATIBLE);
@@ -4601,8 +4594,7 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
 
   @Test public void testInSubquery() {
     check("select * from emp where deptno in (select deptno from dept)");
-    check(
-        "select * from emp where (empno,deptno)"
+    check("select * from emp where (empno,deptno)"
         + " in (select deptno,deptno from dept)");
 
     // NOTE: jhyde: The closing caret should be one character to the right
@@ -4610,7 +4602,7 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     // discarded during the parsing process.
     checkFails(
         "select * from emp where ^deptno in "
-        + "(select deptno,deptno from dept^)",
+            + "(select deptno,deptno from dept^)",
         "Values passed to IN operator must have compatible types");
   }
 
@@ -4645,8 +4637,7 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     check("select 1 from emp as e join emp on emp.empno = e.deptno");
 
     // explicit alias does not clash with overridden alias
-    check(
-        "select 1 from emp as e join dept as emp on e.empno = emp.deptno");
+    check("select 1 from emp as e join dept as emp on e.empno = emp.deptno");
 
     // more than 2 in from clause
     checkFails(
@@ -4673,8 +4664,7 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
         "Duplicate relation name 'EMP' in FROM clause");
 
     // alias does not clash with alias inherited from enclosing context
-    check(
-        "select 1 from emp, dept where exists (\n"
+    check("select 1 from emp, dept where exists (\n"
         + "  select 1 from emp where emp.empno = emp.deptno)");
   }
 
@@ -4710,7 +4700,7 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     // It is an error to refer to a table which has been given another
     // alias.
     checkFails("select * from emp as e where exists (\n"
-        + "  select 1 from dept where dept.deptno = ^emp^.deptno)",
+            + "  select 1 from dept where dept.deptno = ^emp^.deptno)",
         "Table 'EMP' not found");
   }
 
@@ -4718,8 +4708,8 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     // You cannot refer to a table ('e2') in the parent scope of a query in
     // the from clause.
     checkFails("select * from emp as e1 where exists (\n"
-        + "  select * from emp as e2,\n"
-        + "    (select * from dept where dept.deptno = ^e2^.deptno))",
+            + "  select * from emp as e2,\n"
+            + "    (select * from dept where dept.deptno = ^e2^.deptno))",
         "Table 'E2' not found");
   }
 
@@ -4736,43 +4726,43 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
   @Test public void testUnionNameResolution() {
     checkFails(
         "select * from emp as e1 where exists (\n"
-        + "  select * from emp as e2,\n"
-        + "  (select deptno from dept as d\n"
-        + "   union\n"
-        + "   select deptno from emp as e3 where deptno = ^e2^.deptno))",
+            + "  select * from emp as e2,\n"
+            + "  (select deptno from dept as d\n"
+            + "   union\n"
+            + "   select deptno from emp as e3 where deptno = ^e2^.deptno))",
         "Table 'E2' not found");
 
     checkFails("select * from emp\n"
-        + "union\n"
-        + "select * from dept where ^empno^ < 10",
+            + "union\n"
+            + "select * from dept where ^empno^ < 10",
         "Column 'EMPNO' not found in any table");
   }
 
   @Test public void testUnionCountMismatchFails() {
     checkFails(
         "select 1,2 from emp\n"
-        + "union\n"
-        + "select ^3^ from dept",
+            + "union\n"
+            + "select ^3^ from dept",
         "Column count mismatch in UNION");
   }
 
   @Test public void testUnionCountMismatcWithValuesFails() {
     checkFails(
         "select * from ( values (1))\n"
-        + "union\n"
-        + "select ^*^ from ( values (1,2))",
+            + "union\n"
+            + "select ^*^ from ( values (1,2))",
         "Column count mismatch in UNION");
 
     checkFails(
         "select * from ( values (1))\n"
-        + "union\n"
-        + "select ^*^ from emp",
+            + "union\n"
+            + "select ^*^ from emp",
         "Column count mismatch in UNION");
 
     checkFails(
         "select * from emp\n"
-        + "union\n"
-        + "select ^*^ from ( values (1))",
+            + "union\n"
+            + "select ^*^ from ( values (1))",
         "Column count mismatch in UNION");
   }
 
@@ -4799,17 +4789,17 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
   @Test public void testUnionTypeMismatchWithValuesFails() {
     checkFails(
         "values (1, ^2^, 3), (3, 4, 5), (6, 7, 8) union\n"
-        + "select deptno, name, deptno from dept",
+            + "select deptno, name, deptno from dept",
         "Type mismatch in column 2 of UNION");
 
     checkFails(
         "select 1 from (values (^'x'^)) union\n"
-        + "select 'a' from (values ('y'))",
+            + "select 'a' from (values ('y'))",
         "Type mismatch in column 1 of UNION");
 
     checkFails(
         "select 1 from (values (^'x'^)) union\n"
-        + "(values ('a'))",
+            + "(values ('a'))",
         "Type mismatch in column 1 of UNION");
   }
 
@@ -4847,28 +4837,28 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     // USING clause
     checkFails(
         "select * from dept where exists (\n"
-        + "select 1 from emp join bonus using (^dname^))",
+            + "select 1 from emp join bonus using (^dname^))",
         "Column 'DNAME' not found in any table");
 
     // inherited column, found in only one side of the join, in the
     // USING clause
     checkFails(
         "select * from dept where exists (\n"
-        + "select 1 from emp join bonus using (^deptno^))",
+            + "select 1 from emp join bonus using (^deptno^))",
         "Column 'DEPTNO' not found in any table");
   }
 
   @Test public void testCrossJoinOnFails() {
     checkFails(
         "select * from emp cross join dept\n"
-        + " ^on emp.deptno = dept.deptno^",
+            + " ^on emp.deptno = dept.deptno^",
         "Cannot specify condition \\(NATURAL keyword, or ON or USING clause\\) following CROSS JOIN");
   }
 
   @Test public void testInnerJoinWithoutUsingOrOnFails() {
     checkFails(
         "select * from emp inner ^join^ dept\n"
-        + "where emp.deptno = dept.deptno",
+            + "where emp.deptno = dept.deptno",
         "INNER, LEFT, RIGHT or FULL join requires a condition \\(NATURAL keyword or ON or USING clause\\)");
   }
 
@@ -4887,7 +4877,7 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
   @Test public void testNaturalJoinIncompatibleDatatype() {
     checkFails(
         "select * from emp natural ^join^\n"
-        + "(select deptno, name as sal from dept)",
+            + "(select deptno, name as sal from dept)",
         "Column 'SAL' matched using NATURAL keyword or USING clause has incompatible types: cannot compare 'INTEGER' to 'VARCHAR\\(10\\)'");
 
     // make sal occur more than once on rhs, it is ignored and therefore
@@ -4918,44 +4908,44 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     checkResultType(
         "select * from emp left join dept on emp.deptno = dept.deptno",
         "RecordType(INTEGER NOT NULL EMPNO,"
-        + " VARCHAR(20) NOT NULL ENAME,"
-        + " VARCHAR(10) NOT NULL JOB,"
-        + " INTEGER MGR,"
-        + " TIMESTAMP(0) NOT NULL HIREDATE,"
-        + " INTEGER NOT NULL SAL,"
-        + " INTEGER NOT NULL COMM,"
-        + " INTEGER NOT NULL DEPTNO,"
-        + " BOOLEAN NOT NULL SLACKER,"
-        + " INTEGER DEPTNO0,"
-        + " VARCHAR(10) NAME) NOT NULL");
+            + " VARCHAR(20) NOT NULL ENAME,"
+            + " VARCHAR(10) NOT NULL JOB,"
+            + " INTEGER MGR,"
+            + " TIMESTAMP(0) NOT NULL HIREDATE,"
+            + " INTEGER NOT NULL SAL,"
+            + " INTEGER NOT NULL COMM,"
+            + " INTEGER NOT NULL DEPTNO,"
+            + " BOOLEAN NOT NULL SLACKER,"
+            + " INTEGER DEPTNO0,"
+            + " VARCHAR(10) NAME) NOT NULL");
 
     checkResultType(
         "select * from emp right join dept on emp.deptno = dept.deptno",
         "RecordType(INTEGER EMPNO,"
-        + " VARCHAR(20) ENAME,"
-        + " VARCHAR(10) JOB,"
-        + " INTEGER MGR,"
-        + " TIMESTAMP(0) HIREDATE,"
-        + " INTEGER SAL,"
-        + " INTEGER COMM,"
-        + " INTEGER DEPTNO,"
-        + " BOOLEAN SLACKER,"
-        + " INTEGER NOT NULL DEPTNO0,"
-        + " VARCHAR(10) NOT NULL NAME) NOT NULL");
+            + " VARCHAR(20) ENAME,"
+            + " VARCHAR(10) JOB,"
+            + " INTEGER MGR,"
+            + " TIMESTAMP(0) HIREDATE,"
+            + " INTEGER SAL,"
+            + " INTEGER COMM,"
+            + " INTEGER DEPTNO,"
+            + " BOOLEAN SLACKER,"
+            + " INTEGER NOT NULL DEPTNO0,"
+            + " VARCHAR(10) NOT NULL NAME) NOT NULL");
 
     checkResultType(
         "select * from emp full join dept on emp.deptno = dept.deptno",
         "RecordType(INTEGER EMPNO,"
-        + " VARCHAR(20) ENAME,"
-        + " VARCHAR(10) JOB,"
-        + " INTEGER MGR,"
-        + " TIMESTAMP(0) HIREDATE,"
-        + " INTEGER SAL,"
-        + " INTEGER COMM,"
-        + " INTEGER DEPTNO,"
-        + " BOOLEAN SLACKER,"
-        + " INTEGER DEPTNO0,"
-        + " VARCHAR(10) NAME) NOT NULL");
+            + " VARCHAR(20) ENAME,"
+            + " VARCHAR(10) JOB,"
+            + " INTEGER MGR,"
+            + " TIMESTAMP(0) HIREDATE,"
+            + " INTEGER SAL,"
+            + " INTEGER COMM,"
+            + " INTEGER DEPTNO,"
+            + " BOOLEAN SLACKER,"
+            + " INTEGER DEPTNO0,"
+            + " VARCHAR(10) NAME) NOT NULL");
   }
 
   // todo: Cannot handle '(a join b)' yet -- we see the '(' and expect to
@@ -4968,11 +4958,11 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     // counting?)
     checkFails(
         "select * from (emp join bonus using (job)) as x\n"
-        + "join dept using (deptno)",
+            + "join dept using (deptno)",
         "as wrong here");
     checkFails(
         "select * from (emp join bonus using (job))\n"
-        + "join dept using (^dname^)",
+            + "join dept using (^dname^)",
         "dname not found in lhs");
 
     // Needs real Error Message and error marks in query
@@ -4984,21 +4974,20 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
   @Test public void testJoinSubquery() {
     // Sub-queries require alias
     checkFails("select * from (select 1 as one from emp)\n"
-        + "join (values (1), (2)) on true",
+            + "join (values (1), (2)) on true",
         "require alias");
   }
 
   @Test public void testJoinUsingThreeWay() {
-    check(
-        "select *\n"
+    check("select *\n"
         + "from emp as e\n"
         + "join dept as d using (deptno)\n"
         + "join emp as e2 using (empno)");
     checkFails(
         "select *\n"
-        + "from emp as e\n"
-        + "join dept as d using (deptno)\n"
-        + "join dept as d2 using (^deptno^)",
+            + "from emp as e\n"
+            + "join dept as d using (deptno)\n"
+            + "join dept as d2 using (^deptno^)",
         "Column name 'DEPTNO' in USING clause is not unique on one side of join");
   }
 
@@ -5031,8 +5020,8 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
 
   @Test public void testHavingBetween() {
     // FRG-115: having clause with between not working
-    check(
-        "select deptno from emp group by deptno having deptno between 10 and 12");
+    check("select deptno from emp group by deptno\n"
+        + "having deptno between 10 and 12");
 
     // this worked even before FRG-115 was fixed
     check("select deptno from emp group by deptno having deptno + 5 > 10");
@@ -5043,50 +5032,50 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     // simplest possible
     checkResultType(
         "with emp2 as (select * from emp)\n"
-        + "select * from emp2", EMP_RECORD_TYPE);
+            + "select * from emp2", EMP_RECORD_TYPE);
 
     // degree of emp2 column list does not match its query
     checkFails(
         "with emp2 ^(x, y)^ as (select * from emp)\n"
-        + "select * from emp2",
+            + "select * from emp2",
         "Number of columns must match number of query columns");
 
     // duplicate names in column list
     checkFails(
         "with emp2 (x, y, ^y^, x) as (select sal, deptno, ename, empno from emp)\n"
-        + "select * from emp2",
+            + "select * from emp2",
         "Duplicate name 'Y' in column list");
 
     // column list required if aliases are not unique
     checkFails(
         "with emp2 as (^select empno as e, sal, deptno as e from emp^)\n"
-        + "select * from emp2",
+            + "select * from emp2",
         "Column has duplicate column name 'E' and no column list specified");
 
     // forward reference
     checkFails(
         "with emp3 as (select * from ^emp2^),\n"
-        + " emp2 as (select * from emp)\n"
-        + "select * from emp3",
+            + " emp2 as (select * from emp)\n"
+            + "select * from emp3",
         "Table 'EMP2' not found");
 
     // forward reference in with-item not used; should still fail
     checkFails(
         "with emp3 as (select * from ^emp2^),\n"
-        + " emp2 as (select * from emp)\n"
-        + "select * from emp2",
+            + " emp2 as (select * from emp)\n"
+            + "select * from emp2",
         "Table 'EMP2' not found");
 
     // table not used is ok
     checkResultType(
         "with emp2 as (select * from emp),\n"
-        + " emp3 as (select * from emp2)\n"
-        + "select * from emp2",
+            + " emp3 as (select * from emp2)\n"
+            + "select * from emp2",
         EMP_RECORD_TYPE);
 
     // self-reference is not ok, even in table not used
     checkFails("with emp2 as (select * from emp),\n"
-        + " emp3 as (select * from ^emp3^)\n" + "values (1)",
+            + " emp3 as (select * from ^emp3^)\n" + "values (1)",
         "Table 'EMP3' not found");
 
     // self-reference not ok
@@ -5095,9 +5084,9 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
 
     // refer to 2 previous tables, not just immediately preceding
     checkResultType("with emp2 as (select * from emp),\n"
-        + " dept2 as (select * from dept),\n"
-        + " empDept as (select emp2.empno, dept2.deptno from dept2 join emp2 using (deptno))\n"
-        + "select 1 as one from empDept",
+            + " dept2 as (select * from dept),\n"
+            + " empDept as (select emp2.empno, dept2.deptno from dept2 join emp2 using (deptno))\n"
+            + "select 1 as one from empDept",
         "RecordType(INTEGER NOT NULL ONE) NOT NULL");
   }
 
@@ -5106,7 +5095,7 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     // nested WITH (parentheses required - and even with parentheses SQL
     // standard doesn't allow sub-query to have WITH)
     checkResultType("with emp2 as (select * from emp)\n"
-        + "select * from emp2 union all select * from emp",
+            + "select * from emp2 union all select * from emp",
         EMP_RECORD_TYPE);
   }
 
@@ -5135,47 +5124,47 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     // nested WITH (parentheses required - and even with parentheses SQL
     // standard doesn't allow sub-query to have WITH)
     checkResultType("with emp2 as (select * from emp)\n"
-        + "(\n"
-        + "  with dept2 as (select * from dept)\n"
-        + "  (\n"
-        + "    with empDept as (select emp2.empno, dept2.deptno from dept2 join emp2 using (deptno))\n"
-        + "    select 1 as one from empDept))",
+            + "(\n"
+            + "  with dept2 as (select * from dept)\n"
+            + "  (\n"
+            + "    with empDept as (select emp2.empno, dept2.deptno from dept2 join emp2 using (deptno))\n"
+            + "    select 1 as one from empDept))",
         "RecordType(INTEGER NOT NULL ONE) NOT NULL");
 
     // WITH inside WHERE can see enclosing tables
     checkResultType("select * from emp\n"
-        + "where exists (\n"
-        + "  with dept2 as (select * from dept where dept.deptno >= emp.deptno)\n"
-        + "  select 1 from dept2 where deptno <= emp.deptno)",
+            + "where exists (\n"
+            + "  with dept2 as (select * from dept where dept.deptno >= emp.deptno)\n"
+            + "  select 1 from dept2 where deptno <= emp.deptno)",
         EMP_RECORD_TYPE);
 
     // WITH inside FROM cannot see enclosing tables
     checkFails("select * from emp\n"
-        + "join (\n"
-        + "  with dept2 as (select * from dept where dept.deptno >= ^emp^.deptno)\n"
-        + "  select * from dept2) as d on true",
+            + "join (\n"
+            + "  with dept2 as (select * from dept where dept.deptno >= ^emp^.deptno)\n"
+            + "  select * from dept2) as d on true",
         "Table 'EMP' not found");
 
     // as above, using USING
     checkFails("select * from emp\n"
-        + "join (\n"
-        + "  with dept2 as (select * from dept where dept.deptno >= ^emp^.deptno)\n"
-        + "  select * from dept2) as d using (deptno)",
+            + "join (\n"
+            + "  with dept2 as (select * from dept where dept.deptno >= ^emp^.deptno)\n"
+            + "  select * from dept2) as d using (deptno)",
         "Table 'EMP' not found");
 
     // WITH inside FROM
     checkResultType("select e.empno, d.* from emp as e\n"
-        + "join (\n"
-        + "  with dept2 as (select * from dept where dept.deptno > 10)\n"
-        + "  select deptno, 1 as one from dept2) as d using (deptno)",
+            + "join (\n"
+            + "  with dept2 as (select * from dept where dept.deptno > 10)\n"
+            + "  select deptno, 1 as one from dept2) as d using (deptno)",
         "RecordType(INTEGER NOT NULL EMPNO,"
-        + " INTEGER NOT NULL DEPTNO,"
-        + " INTEGER NOT NULL ONE) NOT NULL");
+            + " INTEGER NOT NULL DEPTNO,"
+            + " INTEGER NOT NULL ONE) NOT NULL");
 
     checkFails("select ^e^.empno, d.* from emp\n"
-        + "join (\n"
-        + "  with dept2 as (select * from dept where dept.deptno > 10)\n"
-        + "  select deptno, 1 as one from dept2) as d using (deptno)",
+            + "join (\n"
+            + "  with dept2 as (select * from dept where dept.deptno > 10)\n"
+            + "  select deptno, 1 as one from dept2) as d using (deptno)",
         "Table 'E' not found");
   }
 
@@ -5190,9 +5179,9 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     // SqlOrderBy(SqlUnion(SqlSelect ...)), but is not valid SQL.
     checkFails(
         "select count(*) from emp\n"
-        + "union all\n"
-        + "select count(*) from emp\n"
-        + "order by ^count(*)^",
+            + "union all\n"
+            + "select count(*) from emp\n"
+            + "order by ^count(*)^",
         "Aggregate expression is illegal in ORDER BY clause of non-aggregating SELECT");
   }
 
@@ -5227,8 +5216,7 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     // E.g. "select x1, x2 ... from (
     // select 'a' as x1, 'a' as x2, ... from emp union
     // select 'bb' as x1, 'bb' as x2, ... from dept)"
-    check(
-        "select " + list(", ", "x", x)
+    check("select " + list(", ", "x", x)
         + " from (select " + list(", ", "'a' as x", x) + " from emp "
         + "union all select " + list(", ", "'bb' as x", x) + " from dept)");
   }
@@ -5306,35 +5294,33 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
             // Ambiguous in SQL:2003
             : "col ambig");
 
-    check(
-        "select deptno from dept\n"
+    check("select deptno from dept\n"
         + "union\n"
         + "select empno from emp\n"
         + "order by deptno");
 
     checkFails(
         "select deptno from dept\n"
-        + "union\n"
-        + "select empno from emp\n"
-        + "order by ^empno^",
+            + "union\n"
+            + "select empno from emp\n"
+            + "order by ^empno^",
         "Column 'EMPNO' not found in any table");
 
     checkFails(
         "select deptno from dept\n"
-        + "union\n"
-        + "select empno from emp\n"
-        + "order by ^10^",
+            + "union\n"
+            + "select empno from emp\n"
+            + "order by ^10^",
 
         // invalid in oracle and pre-99
         conformance.isSortByOrdinal() ? "Ordinal out of range" : null);
 
     // Sort by scalar subquery
-    check(
-        "select * from emp\n"
+    check("select * from emp\n"
         + "order by (select name from dept where deptno = emp.deptno)");
     checkFails(
         "select * from emp\n"
-        + "order by (select name from dept where deptno = emp.^foo^)",
+            + "order by (select name from dept where deptno = emp.^foo^)",
         "Column 'FOO' not found in table 'EMP'");
 
     // REVIEW jvs 10-Apr-2008:  I disabled this because I don't
@@ -5359,38 +5345,36 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
   }
 
   @Test public void testOrderUnion() {
-    check(
-        "select empno, sal from emp "
+    check("select empno, sal from emp "
         + "union all "
         + "select deptno, deptno from dept "
         + "order by empno");
 
     checkFails(
         "select empno, sal from emp "
-        + "union all "
-        + "select deptno, deptno from dept "
-        + "order by ^asc^",
+            + "union all "
+            + "select deptno, deptno from dept "
+            + "order by ^asc^",
         "Column 'ASC' not found in any table");
 
     // name belongs to emp but is not projected so cannot sort on it
     checkFails(
         "select empno, sal from emp "
-        + "union all "
-        + "select deptno, deptno from dept "
-        + "order by ^ename^ desc",
+            + "union all "
+            + "select deptno, deptno from dept "
+            + "order by ^ename^ desc",
         "Column 'ENAME' not found in any table");
 
     // empno is not an alias in the first select in the union
     checkFails(
         "select deptno, deptno from dept "
-        + "union all "
-        + "select empno, sal from emp "
-        + "order by deptno asc, ^empno^",
+            + "union all "
+            + "select empno, sal from emp "
+            + "order by deptno asc, ^empno^",
         "Column 'EMPNO' not found in any table");
 
     // ordinals ok
-    check(
-        "select empno, sal from emp "
+    check("select empno, sal from emp "
         + "union all "
         + "select deptno, deptno from dept "
         + "order by 2");
@@ -5400,22 +5384,20 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     if (tester.getConformance().isSortByOrdinal()) {
       checkFails(
           "select empno, sal from emp "
-          + "union all "
-          + "select deptno, deptno from dept "
-          + "order by ^3^",
+              + "union all "
+              + "select deptno, deptno from dept "
+              + "order by ^3^",
           "Ordinal out of range");
     }
 
     // Expressions made up of aliases are OK.
     // (This is illegal in Oracle 10G.)
-    check(
-        "select empno, sal from emp "
+    check("select empno, sal from emp "
         + "union all "
         + "select deptno, deptno from dept "
         + "order by empno * sal + 2");
 
-    check(
-        "select empno, sal from emp "
+    check("select empno, sal from emp "
         + "union all "
         + "select deptno, deptno from dept "
         + "order by 'foobar'");
@@ -5431,8 +5413,7 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
         "Expression 'EMP\\.EMPNO' is not being grouped");
 
     // order by can contain aggregate expressions
-    check(
-        "select empno from emp "
+    check("select empno from emp "
         + "group by empno, deptno "
         + "order by deptno * sum(sal + 2)");
 
@@ -5442,8 +5423,7 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
         "select sum(sal) from emp having count(*) > 3 order by ^empno^",
         "Expression 'EMP\\.EMPNO' is not being grouped");
 
-    check(
-        "select sum(sal) from emp having count(*) > 3 order by sum(deptno)");
+    check("select sum(sal) from emp having count(*) > 3 order by sum(deptno)");
 
     // Select distinct
 
@@ -5455,47 +5435,44 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
         "select distinct deptno from emp group by deptno order by deptno, ^empno^",
         "Expression 'EMP\\.EMPNO' is not in the select clause");
 
-    check(
-        "select distinct deptno from emp group by deptno order by deptno");
+    check("select distinct deptno from emp group by deptno order by deptno");
 
     // UNION of SELECT DISTINCT and GROUP BY behaves just like a UNION.
-    check(
-        "select distinct deptno from dept "
+    check("select distinct deptno from dept "
         + "union all "
         + "select empno from emp group by deptno, empno "
         + "order by deptno");
 
     // order by can contain a mixture of aliases and aggregate expressions
-    check(
-        "select empno as x "
+    check("select empno as x "
         + "from emp "
         + "group by empno, deptno "
         + "order by x * sum(sal + 2)");
 
     checkFails(
         "select empno as x "
-        + "from emp "
-        + "group by empno, deptno "
-        + "order by empno * sum(sal + 2)",
+            + "from emp "
+            + "group by empno, deptno "
+            + "order by empno * sum(sal + 2)",
         tester.getConformance().isSortByAliasObscures() ? "xxxx" : null);
 
     // Distinct on expressions with attempts to order on a column in
     // the underlying table
     checkFails(
         "select distinct cast(empno as bigint) "
-        + "from emp order by ^empno^",
+            + "from emp order by ^empno^",
         "Expression 'EMP\\.EMPNO' is not in the select clause");
     checkFails(
         "select distinct cast(empno as bigint) "
-        + "from emp order by ^emp.empno^",
+            + "from emp order by ^emp.empno^",
         "Expression 'EMP\\.EMPNO' is not in the select clause");
     checkFails(
         "select distinct cast(empno as bigint) as empno "
-        + "from emp order by ^emp.empno^",
+            + "from emp order by ^emp.empno^",
         "Expression 'EMP\\.EMPNO' is not in the select clause");
     checkFails(
         "select distinct cast(empno as bigint) as empno "
-        + "from emp as e order by ^e.empno^",
+            + "from emp as e order by ^e.empno^",
         "Expression 'E\\.EMPNO' is not in the select clause");
 
     // These tests are primarily intended to test cases where sorting by
@@ -5503,39 +5480,35 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     // by alias, the tests also verify that a proper exception is thrown.
     checkFails(
         "select distinct cast(empno as bigint) as empno "
-        + "from emp order by ^empno^",
+            + "from emp order by ^empno^",
         tester.getConformance().isSortByAlias() ? null
             : "Expression 'EMPNO' is not in the select clause");
     checkFails(
         "select distinct cast(empno as bigint) as eno "
-        + "from emp order by ^eno^",
+            + "from emp order by ^eno^",
         tester.getConformance().isSortByAlias() ? null
             : "Column 'ENO' not found in any table");
     checkFails(
         "select distinct cast(empno as bigint) as empno "
-        + "from emp e order by ^empno^",
+            + "from emp e order by ^empno^",
         tester.getConformance().isSortByAlias() ? null
             : "Expression 'EMPNO' is not in the select clause");
 
     // Distinct on expressions, sorting using ordinals.
     if (tester.getConformance().isSortByOrdinal()) {
-      check(
-          "select distinct cast(empno as bigint) from emp order by 1");
-      check(
-          "select distinct cast(empno as bigint) as empno "
+      check("select distinct cast(empno as bigint) from emp order by 1");
+      check("select distinct cast(empno as bigint) as empno "
           + "from emp order by 1");
-      check(
-          "select distinct cast(empno as bigint) as empno "
+      check("select distinct cast(empno as bigint) as empno "
           + "from emp as e order by 1");
     }
 
     // Distinct on expressions with ordering on expressions as well
-    check(
-        "select distinct cast(empno as varchar(10)) from emp "
+    check("select distinct cast(empno as varchar(10)) from emp "
         + "order by cast(empno as varchar(10))");
     checkFails(
         "select distinct cast(empno as varchar(10)) as eno from emp "
-        + " order by upper(^eno^)",
+            + " order by upper(^eno^)",
         tester.getConformance().isSortByAlias() ? null
             : "Column 'ENO' not found in any table");
   }
@@ -5555,22 +5528,19 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
 
     // If we're grouping on ALL columns, 'select *' is ok.
     // Checked on Oracle10G.
-    check(
-        "select * from (select empno,deptno from emp) group by deptno,empno");
+    check("select * from (select empno,deptno from emp) group by deptno,empno");
 
     // This query tries to reference an agg expression from within a
     // subquery as a correlating expression, but the SQL syntax rules say
     // that the agg function SUM always applies to the current scope.
     // As it happens, the query is valid.
-    check(
-        "select deptno\n"
+    check("select deptno\n"
         + "from emp\n"
         + "group by deptno\n"
         + "having exists (select sum(emp.sal) > 10 from (values(true)))");
 
     // if you reference a column from a subquery, it must be a group col
-    check(
-        "select deptno "
+    check("select deptno "
         + "from emp "
         + "group by deptno "
         + "having exists (select 1 from (values(true)) where emp.deptno = 10)");
@@ -5579,9 +5549,9 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     if (TODO) {
       checkFails(
           "select deptno "
-          + "from emp "
-          + "group by deptno "
-          + "having exists (select 1 from (values(true)) where emp.empno = 10)",
+              + "from emp "
+              + "group by deptno "
+              + "having exists (select 1 from (values(true)) where emp.empno = 10)",
           "xx");
     }
 
@@ -5595,8 +5565,8 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     // -- see sql2003 part2,  7.9
     checkFails(
         "select count(*)\n"
-        + "from emp\n"
-        + "where exists (select count(*) from dept group by ^emp^.empno)",
+            + "from emp\n"
+            + "where exists (select count(*) from dept group by ^emp^.empno)",
         "Table 'EMP' not found");
   }
 
@@ -5608,8 +5578,8 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
         "Expression 'EMPNO' is not being grouped");
 
     // datatype equivalence
-    check(
-        "select cast(empno as VARCHAR(10)) from emp group by cast(empno as VARCHAR(10))");
+    check("select cast(empno as VARCHAR(10)) from emp\n"
+        + "group by cast(empno as VARCHAR(10))");
     checkFails(
         "select cast(^empno^ as VARCHAR(11)) from emp group by cast(empno as VARCHAR(10))",
         "Expression 'EMPNO' is not being grouped");
@@ -5617,27 +5587,23 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
 
   @Test public void testGroupExpressionEquivalenceId() {
     // identifier equivalence
-    check(
-        "select case empno when 10 then deptno else null end from emp "
+    check("select case empno when 10 then deptno else null end from emp "
         + "group by case empno when 10 then deptno else null end");
 
     // matches even when one column is qualified (checked on Oracle10.1)
-    check(
-        "select case empno when 10 then deptno else null end from emp "
+    check("select case empno when 10 then deptno else null end from emp "
         + "group by case empno when 10 then emp.deptno else null end");
-    check(
-        "select case empno when 10 then deptno else null end from emp "
+    check("select case empno when 10 then deptno else null end from emp "
         + "group by case emp.empno when 10 then emp.deptno else null end");
-    check(
-        "select case emp.empno when 10 then deptno else null end from emp "
+    check("select case emp.empno when 10 then deptno else null end from emp "
         + "group by case empno when 10 then emp.deptno else null end");
 
     // emp.deptno is different to dept.deptno (even though there is an '='
     // between them)
     checkFails(
         "select case ^emp.empno^ when 10 then emp.deptno else null end "
-        + "from emp join dept on emp.deptno = dept.deptno "
-        + "group by case emp.empno when 10 then dept.deptno else null end",
+            + "from emp join dept on emp.deptno = dept.deptno "
+            + "group by case emp.empno when 10 then dept.deptno else null end",
         "Expression 'EMP\\.EMPNO' is not being grouped");
   }
 
@@ -5645,11 +5611,9 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
   public void _testGroupExpressionEquivalenceCorrelated() {
     // dname comes from dept, so it is constant within the subquery, and
     // is so is a valid expr in a group-by query
-    check(
-        "select * from dept where exists ("
+    check("select * from dept where exists ("
         + "select dname from emp group by empno)");
-    check(
-        "select * from dept where exists ("
+    check("select * from dept where exists ("
         + "select dname + empno + 1 from emp group by empno, dept.deptno)");
   }
 
@@ -5666,56 +5630,56 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     // CASE expression.
 
     // literal equivalence
-    check(
-        "select case empno when 10 then date '1969-04-29' else null end from emp "
+    check("select case empno when 10 then date '1969-04-29' else null end\n"
+        + "from emp\n"
         + "group by case empno when 10 then date '1969-04-29' else null end");
 
     // this query succeeds in oracle 10.1 because 1 and 1.0 have the same
     // type
     checkFails(
         "select case ^empno^ when 10 then 1 else null end from emp "
-        + "group by case empno when 10 then 1.0 else null end",
+            + "group by case empno when 10 then 1.0 else null end",
         "Expression 'EMPNO' is not being grouped");
 
     // 3.1415 and 3.14150 are different literals (I don't care either way)
     checkFails(
         "select case ^empno^ when 10 then 3.1415 else null end from emp "
-        + "group by case empno when 10 then 3.14150 else null end",
+            + "group by case empno when 10 then 3.14150 else null end",
         "Expression 'EMPNO' is not being grouped");
 
     // 3 and 03 are the same literal (I don't care either way)
-    check(
-        "select case empno when 10 then 03 else null end from emp "
+    check("select case empno when 10 then 03 else null end from emp "
         + "group by case empno when 10 then 3 else null end");
     checkFails(
         "select case ^empno^ when 10 then 1 else null end from emp "
-        + "group by case empno when 10 then 2 else null end",
+            + "group by case empno when 10 then 2 else null end",
         "Expression 'EMPNO' is not being grouped");
-    check(
-        "select case empno when 10 then timestamp '1969-04-29 12:34:56.0' else null end from emp "
+    check("select case empno when 10 then timestamp '1969-04-29 12:34:56.0'\n"
+        + "       else null end from emp\n"
         + "group by case empno when 10 then timestamp '1969-04-29 12:34:56' else null end");
   }
 
   @Test public void testGroupExpressionEquivalenceStringLiteral() {
-    check(
-        "select case empno when 10 then 'foo bar' else null end from emp "
+    check("select case empno when 10 then 'foo bar' else null end from emp "
         + "group by case empno when 10 then 'foo bar' else null end");
 
     if (Bug.FRG78_FIXED) {
-      check(
-          "select case empno when 10 then _iso-8859-1'foo bar' collate latin1$en$1 else null end from emp "
-          + "group by case empno when 10 then _iso-8859-1'foo bar' collate latin1$en$1 else null end");
+      check("select case empno when 10\n"
+          + "      then _iso-8859-1'foo bar' collate latin1$en$1 else null end\n"
+          + "from emp\n"
+          + "group by case empno when 10\n"
+          + "      then _iso-8859-1'foo bar' collate latin1$en$1 else null end");
     }
 
     checkFails(
         "select case ^empno^ when 10 then _iso-8859-1'foo bar' else null end from emp "
-        + "group by case empno when 10 then _UTF16'foo bar' else null end",
+            + "group by case empno when 10 then _UTF16'foo bar' else null end",
         "Expression 'EMPNO' is not being grouped");
 
     if (Bug.FRG78_FIXED) {
       checkFails(
           "select case ^empno^ when 10 then 'foo bar' collate latin1$en$1 else null end from emp "
-          + "group by case empno when 10 then 'foo bar' collate latin1$fr$1 else null end",
+              + "group by case empno when 10 then 'foo bar' collate latin1$fr$1 else null end",
           "Expression 'EMPNO' is not being grouped");
     }
   }
@@ -5744,7 +5708,7 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     // make sure it fails in HAVING too
     checkFails(
         "select count(*) from emp group by deptno "
-        + "having ^sum(max(empno))^=3",
+            + "having ^sum(max(empno))^=3",
         ERR_NESTED_AGG);
 
     // double-nesting should fail too; bottom-up validation currently
@@ -5797,13 +5761,11 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
 
   @Test public void testCorrelatingVariables() {
     // reference to unqualified correlating column
-    check(
-        "select * from emp where exists (\n"
+    check("select * from emp where exists (\n"
         + "select * from dept where deptno = sal)");
 
     // reference to qualified correlating column
-    check(
-        "select * from emp where exists (\n"
+    check("select * from emp where exists (\n"
         + "select * from dept where deptno = emp.sal)");
   }
 
@@ -5933,12 +5895,11 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
       check("select case when deptno = ? then 1 else 2 end from emp");
     }
     if (TODO_TYPE_INFERENCE) {
-      check(
-          "select deptno from emp group by substring(name from ? for ?)");
+      check("select deptno from emp group by substring(name from ? for ?)");
     }
     if (TODO_TYPE_INFERENCE) {
-      check(
-          "select deptno from emp group by case when deptno = ? then 1 else 2 end");
+      check("select deptno from emp\n"
+          + "group by case when deptno = ? then 1 else 2 end");
     }
     check("select 1 from emp having sum(sal) < ?");
   }
@@ -5979,8 +5940,7 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
   }
 
   @Test public void testCorrelationJoin() {
-    check(
-        "select *,"
+    check("select *,"
         + "         multiset(select * from emp where deptno=dept.deptno) "
         + "               as empset"
         + "      from dept");
@@ -6008,12 +5968,12 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
         "select * from emp, (select * from dept where ^emp^.deptno=dept.deptno)",
         "Table 'EMP' not found");
 
-    check(
-        "select * from emp, LATERAL (select * from dept where emp.deptno=dept.deptno)");
-    check(
-        "select * from emp, LATERAL (select * from dept where emp.deptno=dept.deptno) as ldt");
-    check(
-        "select * from emp, LATERAL (select * from dept where emp.deptno=dept.deptno) ldt");
+    check("select * from emp,\n"
+        + "  LATERAL (select * from dept where emp.deptno=dept.deptno)");
+    check("select * from emp,\n"
+        + "  LATERAL (select * from dept where emp.deptno=dept.deptno) as ldt");
+    check("select * from emp,\n"
+        + "  LATERAL (select * from dept where emp.deptno=dept.deptno) ldt");
   }
 
   @Test public void testCollect() {
@@ -6096,8 +6056,8 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
         "Expression 'DEPTNO' is not being grouped");
     check("SELECT DISTINCT deptno, sal from emp GROUP BY sal, deptno");
     check("SELECT deptno FROM emp GROUP BY deptno HAVING deptno > 55");
-    check(
-        "SELECT DISTINCT deptno, 33 FROM emp GROUP BY deptno HAVING deptno > 55");
+    check("SELECT DISTINCT deptno, 33 FROM emp\n"
+        + "GROUP BY deptno HAVING deptno > 55");
     checkFails(
         "SELECT DISTINCT deptno, 33 FROM emp HAVING ^deptno^ > 55",
         "Expression 'DEPTNO' is not being grouped");
@@ -6128,8 +6088,8 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
         "Expression 'EMP\\.EMPNO' is not in the select clause");
 
     // redundant distinct; same query is in unitsql/optimizer/distinct.sql
-    check(
-        "select distinct * from (select distinct deptno from emp) order by 1");
+    check("select distinct * from (\n"
+        + "  select distinct deptno from emp) order by 1");
 
     check("SELECT DISTINCT 5, 10+5, 'string' from emp");
   }
@@ -6137,14 +6097,14 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
   @Test public void testExplicitTable() {
     final String empRecordType =
         "RecordType(INTEGER NOT NULL EMPNO,"
-        + " VARCHAR(20) NOT NULL ENAME,"
-        + " VARCHAR(10) NOT NULL JOB,"
-        + " INTEGER MGR,"
-        + " TIMESTAMP(0) NOT NULL HIREDATE,"
-        + " INTEGER NOT NULL SAL,"
-        + " INTEGER NOT NULL COMM,"
-        + " INTEGER NOT NULL DEPTNO,"
-        + " BOOLEAN NOT NULL SLACKER) NOT NULL";
+            + " VARCHAR(20) NOT NULL ENAME,"
+            + " VARCHAR(10) NOT NULL JOB,"
+            + " INTEGER MGR,"
+            + " TIMESTAMP(0) NOT NULL HIREDATE,"
+            + " INTEGER NOT NULL SAL,"
+            + " INTEGER NOT NULL COMM,"
+            + " INTEGER NOT NULL DEPTNO,"
+            + " BOOLEAN NOT NULL SLACKER) NOT NULL";
     checkResultType("select * from (table emp)", empRecordType);
     checkResultType("table emp", empRecordType);
     checkFails(
@@ -6204,8 +6164,7 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
   public void _testSubqueryInOnClause() {
     // Currently not supported. Should give validator error, but gives
     // internal error.
-    check(
-        "select * from emp as emps left outer join dept as depts\n"
+    check("select * from emp as emps left outer join dept as depts\n"
         + "on emps.deptno = depts.deptno and emps.deptno = (\n"
         + "select min(deptno) from dept as depts2)");
   }
@@ -6235,8 +6194,7 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     check("SELECT * FROM emp TABLESAMPLE SYSTEM(50)");
 
     // applied to query
-    check(
-        "SELECT * FROM ("
+    check("SELECT * FROM ("
         + "SELECT deptno FROM emp "
         + "UNION ALL "
         + "SELECT deptno FROM dept) AS x TABLESAMPLE SUBSTITUTE('foo') "
@@ -6244,20 +6202,18 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
 
     checkFails(
         "SELECT x.^empno^ FROM ("
-        + "SELECT deptno FROM emp TABLESAMPLE SUBSTITUTE('bar') "
-        + "UNION ALL "
-        + "SELECT deptno FROM dept) AS x TABLESAMPLE SUBSTITUTE('foo') "
-        + "ORDER BY 1",
+            + "SELECT deptno FROM emp TABLESAMPLE SUBSTITUTE('bar') "
+            + "UNION ALL "
+            + "SELECT deptno FROM dept) AS x TABLESAMPLE SUBSTITUTE('foo') "
+            + "ORDER BY 1",
         "Column 'EMPNO' not found in table 'X'");
 
-    check(
-        "select * from (\n"
+    check("select * from (\n"
         + "    select * from emp\n"
         + "    join dept on emp.deptno = dept.deptno\n"
         + ") tablesample substitute('SMALL')");
 
-    check(
-        "SELECT * FROM ("
+    check("SELECT * FROM ("
         + "SELECT deptno FROM emp "
         + "UNION ALL "
         + "SELECT deptno FROM dept) AS x TABLESAMPLE BERNOULLI(50) "
@@ -6265,20 +6221,18 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
 
     checkFails(
         "SELECT x.^empno^ FROM ("
-        + "SELECT deptno FROM emp TABLESAMPLE BERNOULLI(50) "
-        + "UNION ALL "
-        + "SELECT deptno FROM dept) AS x TABLESAMPLE BERNOULLI(10) "
-        + "ORDER BY 1",
+            + "SELECT deptno FROM emp TABLESAMPLE BERNOULLI(50) "
+            + "UNION ALL "
+            + "SELECT deptno FROM dept) AS x TABLESAMPLE BERNOULLI(10) "
+            + "ORDER BY 1",
         "Column 'EMPNO' not found in table 'X'");
 
-    check(
-        "select * from (\n"
+    check("select * from (\n"
         + "    select * from emp\n"
         + "    join dept on emp.deptno = dept.deptno\n"
         + ") tablesample bernoulli(10)");
 
-    check(
-        "SELECT * FROM ("
+    check("SELECT * FROM ("
         + "SELECT deptno FROM emp "
         + "UNION ALL "
         + "SELECT deptno FROM dept) AS x TABLESAMPLE SYSTEM(50) "
@@ -6286,14 +6240,13 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
 
     checkFails(
         "SELECT x.^empno^ FROM ("
-        + "SELECT deptno FROM emp TABLESAMPLE SYSTEM(50) "
-        + "UNION ALL "
-        + "SELECT deptno FROM dept) AS x TABLESAMPLE SYSTEM(10) "
-        + "ORDER BY 1",
+            + "SELECT deptno FROM emp TABLESAMPLE SYSTEM(50) "
+            + "UNION ALL "
+            + "SELECT deptno FROM dept) AS x TABLESAMPLE SYSTEM(10) "
+            + "ORDER BY 1",
         "Column 'EMPNO' not found in table 'X'");
 
-    check(
-        "select * from (\n"
+    check("select * from (\n"
         + "    select * from emp\n"
         + "    join dept on emp.deptno = dept.deptno\n"
         + ") tablesample system(10)");
@@ -6306,7 +6259,7 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
         validator,
         "select * from dept",
         "SELECT *\n"
-        + "FROM `DEPT`");
+            + "FROM `DEPT`");
   }
 
   @Test public void testRewriteWithIdentifierExpansion() {
@@ -6316,7 +6269,7 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
         validator,
         "select * from dept",
         "SELECT `DEPT`.`DEPTNO`, `DEPT`.`NAME`\n"
-        + "FROM `CATALOG`.`SALES`.`DEPT` AS `DEPT`");
+            + "FROM `CATALOG`.`SALES`.`DEPT` AS `DEPT`");
   }
 
   @Test public void testRewriteWithColumnReferenceExpansion() {
@@ -6330,13 +6283,13 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     tester.checkRewrite(
         validator,
         "select name from dept where name = 'Moonracer' group by name"
-        + " having sum(deptno) > 3 order by name",
+            + " having sum(deptno) > 3 order by name",
         "SELECT `DEPT`.`NAME`\n"
-        + "FROM `CATALOG`.`SALES`.`DEPT` AS `DEPT`\n"
-        + "WHERE `DEPT`.`NAME` = 'Moonracer'\n"
-        + "GROUP BY `DEPT`.`NAME`\n"
-        + "HAVING SUM(`DEPT`.`DEPTNO`) > 3\n"
-        + "ORDER BY `NAME`");
+            + "FROM `CATALOG`.`SALES`.`DEPT` AS `DEPT`\n"
+            + "WHERE `DEPT`.`NAME` = 'Moonracer'\n"
+            + "GROUP BY `DEPT`.`NAME`\n"
+            + "HAVING SUM(`DEPT`.`DEPTNO`) > 3\n"
+            + "ORDER BY `NAME`");
   }
 
   @Test public void testRewriteWithColumnReferenceExpansionAndFromAlias() {
@@ -6350,15 +6303,15 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     tester.checkRewrite(
         validator,
         "select name from (select * from dept)"
-        + " where name = 'Moonracer' group by name"
-        + " having sum(deptno) > 3 order by name",
+            + " where name = 'Moonracer' group by name"
+            + " having sum(deptno) > 3 order by name",
         "SELECT `EXPR$0`.`NAME`\n"
-        + "FROM (SELECT `DEPT`.`DEPTNO`, `DEPT`.`NAME`\n"
-        + "FROM `CATALOG`.`SALES`.`DEPT` AS `DEPT`) AS `EXPR$0`\n"
-        + "WHERE `EXPR$0`.`NAME` = 'Moonracer'\n"
-        + "GROUP BY `EXPR$0`.`NAME`\n"
-        + "HAVING SUM(`EXPR$0`.`DEPTNO`) > 3\n"
-        + "ORDER BY `NAME`");
+            + "FROM (SELECT `DEPT`.`DEPTNO`, `DEPT`.`NAME`\n"
+            + "FROM `CATALOG`.`SALES`.`DEPT` AS `DEPT`) AS `EXPR$0`\n"
+            + "WHERE `EXPR$0`.`NAME` = 'Moonracer'\n"
+            + "GROUP BY `EXPR$0`.`NAME`\n"
+            + "HAVING SUM(`EXPR$0`.`DEPTNO`) > 3\n"
+            + "ORDER BY `NAME`");
   }
 
   @Test public void testCoalesceWithoutRewrite() {
@@ -6369,13 +6322,13 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
           validator,
           "select coalesce(deptno, empno) from emp",
           "SELECT COALESCE(`EMP`.`DEPTNO`, `EMP`.`EMPNO`)\n"
-          + "FROM `CATALOG`.`SALES`.`EMP` AS `EMP`");
+              + "FROM `CATALOG`.`SALES`.`EMP` AS `EMP`");
     } else {
       tester.checkRewrite(
           validator,
           "select coalesce(deptno, empno) from emp",
           "SELECT COALESCE(`DEPTNO`, `EMPNO`)\n"
-          + "FROM `EMP`");
+              + "FROM `EMP`");
     }
   }
 
@@ -6387,13 +6340,13 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
           validator,
           "select coalesce(deptno, empno) from emp",
           "SELECT CASE WHEN `EMP`.`DEPTNO` IS NOT NULL THEN `EMP`.`DEPTNO` ELSE `EMP`.`EMPNO` END\n"
-          + "FROM `CATALOG`.`SALES`.`EMP` AS `EMP`");
+              + "FROM `CATALOG`.`SALES`.`EMP` AS `EMP`");
     } else {
       tester.checkRewrite(
           validator,
           "select coalesce(deptno, empno) from emp",
           "SELECT CASE WHEN `DEPTNO` IS NOT NULL THEN `DEPTNO` ELSE `EMPNO` END\n"
-          + "FROM `EMP`");
+              + "FROM `EMP`");
     }
   }
 
@@ -6407,25 +6360,25 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     tester.checkFieldOrigin(
         "select * from emp join dept on true",
         "{CATALOG.SALES.EMP.EMPNO,"
-        + " CATALOG.SALES.EMP.ENAME,"
-        + " CATALOG.SALES.EMP.JOB,"
-        + " CATALOG.SALES.EMP.MGR,"
-        + " CATALOG.SALES.EMP.HIREDATE,"
-        + " CATALOG.SALES.EMP.SAL,"
-        + " CATALOG.SALES.EMP.COMM,"
-        + " CATALOG.SALES.EMP.DEPTNO,"
-        + " CATALOG.SALES.EMP.SLACKER,"
-        + " CATALOG.SALES.DEPT.DEPTNO,"
-        + " CATALOG.SALES.DEPT.NAME}");
+            + " CATALOG.SALES.EMP.ENAME,"
+            + " CATALOG.SALES.EMP.JOB,"
+            + " CATALOG.SALES.EMP.MGR,"
+            + " CATALOG.SALES.EMP.HIREDATE,"
+            + " CATALOG.SALES.EMP.SAL,"
+            + " CATALOG.SALES.EMP.COMM,"
+            + " CATALOG.SALES.EMP.DEPTNO,"
+            + " CATALOG.SALES.EMP.SLACKER,"
+            + " CATALOG.SALES.DEPT.DEPTNO,"
+            + " CATALOG.SALES.DEPT.NAME}");
 
     tester.checkFieldOrigin(
         "select distinct emp.empno, hiredate, 1 as one,\n"
-        + " emp.empno * 2 as twiceEmpno\n"
-        + "from emp join dept on true",
+            + " emp.empno * 2 as twiceEmpno\n"
+            + "from emp join dept on true",
         "{CATALOG.SALES.EMP.EMPNO,"
-        + " CATALOG.SALES.EMP.HIREDATE,"
-        + " null,"
-        + " null}");
+            + " CATALOG.SALES.EMP.HIREDATE,"
+            + " null,"
+            + " null}");
   }
 
   @Test public void testBrackets() {
@@ -6440,7 +6393,7 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
 
     tester1.checkQueryFails(
         "select ^x^ from (\n"
-        + "  select [e].EMPNO as [x] from [EMP] as [e])",
+            + "  select [e].EMPNO as [x] from [EMP] as [e])",
         "Column 'X' not found in any table");
 
     tester1.checkQueryFails(
@@ -6449,7 +6402,7 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
 
     tester1.checkResultType(
         "select [x[y]] z ] from (\n"
-        + "  select [e].EMPNO as [x[y]] z ] from [EMP] as [e])",
+            + "  select [e].EMPNO as [x[y]] z ] from [EMP] as [e])",
         "RecordType(INTEGER NOT NULL x[y] z ) NOT NULL");
   }
 
@@ -6469,7 +6422,7 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
 
     tester1.checkQueryFails(
         "select ^x^ from (\n"
-        + "  select e.EMPNO as X from EMP as e)",
+            + "  select e.EMPNO as X from EMP as e)",
         "Column 'x' not found in any table");
 
     // double-quotes are not valid in this lexical convention
@@ -6481,7 +6434,7 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     // can use back-ticks if you really have to
     tester1.checkResultType(
         "select `x[y] z ` from (\n"
-        + "  select e.EMPNO as `x[y] z ` from EMP as e)",
+            + "  select e.EMPNO as `x[y] z ` from EMP as e)",
         "RecordType(INTEGER NOT NULL x[y] z ) NOT NULL");
   }
 
@@ -6528,7 +6481,7 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     // can use double-quote if you really have to
     tester1.checkResultType(
         "select \"x[y] z \" from (\n"
-        + "  select e.EMPNO as \"x[y] z \" from EMP as e)",
+            + "  select e.EMPNO as \"x[y] z \" from EMP as e)",
         "RecordType(INTEGER NOT NULL x[y] z ) NOT NULL");
   }
 
@@ -6549,10 +6502,10 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     // correlating variable
     tester1.checkQuery(
         "select * from emp as [e] where exists (\n"
-        + "select 1 from dept where dept.deptno = [E].deptno)");
+            + "select 1 from dept where dept.deptno = [E].deptno)");
     tester2.checkQueryFails(
         "select * from emp as [e] where exists (\n"
-        + "select 1 from dept where dept.deptno = ^[E]^.deptno)",
+            + "select 1 from dept where dept.deptno = ^[E]^.deptno)",
         "(?s).*Table 'E' not found");
 
     checkFails("select count(1), ^empno^ from emp",
@@ -6644,7 +6597,7 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
         .withCaseSensitive(false)
         .withQuoting(Quoting.BRACKET);
     tester1.checkQueryFails("insert into EMP ([EMPNO], deptno, ^[empno]^)\n"
-        + " values (1, 1, 1)",
+            + " values (1, 1, 1)",
         "Target column 'EMPNO' is assigned more than once");
   }
 


[37/58] [abbrv] [partial] incubator-calcite git commit: [CALCITE-306] Standardize code style for "import package.*; "

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/core/Intersect.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/Intersect.java b/core/src/main/java/org/apache/calcite/rel/core/Intersect.java
index 6f43441..472cd92 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/Intersect.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/Intersect.java
@@ -14,25 +14,30 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel;
+package org.apache.calcite.rel.core;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelInput;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.sql.SqlKind;
 
 import java.util.BitSet;
 import java.util.List;
 
-import org.eigenbase.rel.metadata.RelMetadataQuery;
-import org.eigenbase.relopt.RelOptCluster;
-import org.eigenbase.relopt.RelTraitSet;
-import org.eigenbase.sql.SqlKind;
-
 /**
- * Abstract base class for implementations of
- * {@link IntersectRel}.
+ * Relational expression that returns the intersection of the rows of its
+ * inputs.
+ *
+ * <p>If "all" is true, performs then multiset intersection; otherwise,
+ * performs set set intersection (implying no duplicates in the results).
  */
-public abstract class IntersectRelBase extends SetOpRel {
+public abstract class Intersect extends SetOp {
   /**
-   * Creates an IntersectRelBase.
+   * Creates an Intersect.
    */
-  public IntersectRelBase(
+  public Intersect(
       RelOptCluster cluster,
       RelTraitSet traits,
       List<RelNode> inputs,
@@ -41,14 +46,13 @@ public abstract class IntersectRelBase extends SetOpRel {
   }
 
   /**
-   * Creates an IntersectRelBase by parsing serialized output.
+   * Creates an Intersect by parsing serialized output.
    */
-  protected IntersectRelBase(RelInput input) {
+  protected Intersect(RelInput input) {
     super(input);
   }
 
-  @Override
-  public double getRows() {
+  @Override public double getRows() {
     // REVIEW jvs 30-May-2005:  I just pulled this out of a hat.
     double dRows = Double.MAX_VALUE;
     for (RelNode input : inputs) {
@@ -59,8 +63,7 @@ public abstract class IntersectRelBase extends SetOpRel {
     return dRows;
   }
 
-  @Override
-  public boolean isKey(BitSet columns) {
+  @Override public boolean isKey(BitSet columns) {
     for (RelNode input : inputs) {
       if (input.isKey(columns)) {
         return true;
@@ -70,4 +73,4 @@ public abstract class IntersectRelBase extends SetOpRel {
   }
 }
 
-// End IntersectRelBase.java
+// End Intersect.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 e37c196..c3b2f11 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
@@ -14,27 +14,44 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel;
-
-import java.util.*;
-
-import org.eigenbase.rel.metadata.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.rex.*;
-import org.eigenbase.sql.type.SqlTypeName;
-import org.eigenbase.util.*;
-
-import net.hydromatic.optiq.runtime.FlatLists;
+package org.apache.calcite.rel.core;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.AbstractRelNode;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelVisitor;
+import org.apache.calcite.rel.RelWriter;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexChecker;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.runtime.FlatLists;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
 
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
 /**
- * <code>JoinRelBase</code> is an abstract base class for implementations of
- * {@link JoinRel}.
+ * Relational expression that combines two relational expressions according to
+ * some condition.
+ *
+ * <p>Each output row has columns from the left and right inputs.
+ * The set of output rows is a subset of the cartesian product of the two
+ * inputs; precisely which subset depends on the join condition.
  */
-public abstract class JoinRelBase extends AbstractRelNode {
+public abstract class Join extends AbstractRelNode {
   //~ Instance fields --------------------------------------------------------
 
   protected final RexNode condition;
@@ -43,15 +60,15 @@ public abstract class JoinRelBase extends AbstractRelNode {
   protected final ImmutableSet<String> variablesStopped;
 
   /**
-   * Values must be of enumeration {@link JoinRelType}, except that {@link
-   * JoinRelType#RIGHT} is disallowed.
+   * Values must be of enumeration {@link JoinRelType}, except that
+   * {@link JoinRelType#RIGHT} is disallowed.
    */
   protected JoinRelType joinType;
 
   //~ Constructors -----------------------------------------------------------
 
   /**
-   * Creates a JoinRelBase.
+   * Creates a Join.
    *
    * @param cluster          Cluster
    * @param traits           Traits
@@ -61,9 +78,9 @@ public abstract class JoinRelBase extends AbstractRelNode {
    * @param joinType         Join type
    * @param variablesStopped Set of names of variables which are set by the
    *                         LHS and used by the RHS and are not available to
-   *                         nodes above this JoinRel in the tree
+   *                         nodes above this LogicalJoin in the tree
    */
-  protected JoinRelBase(
+  protected Join(
       RelOptCluster cluster,
       RelTraitSet traits,
       RelNode left,
@@ -83,8 +100,7 @@ public abstract class JoinRelBase extends AbstractRelNode {
 
   //~ Methods ----------------------------------------------------------------
 
-  @Override
-  public List<RexNode> getChildExps() {
+  @Override public List<RexNode> getChildExps() {
     return ImmutableList.of(condition);
   }
 
@@ -157,7 +173,7 @@ public abstract class JoinRelBase extends AbstractRelNode {
   }
 
   public static double estimateJoinedRows(
-      JoinRelBase joinRel,
+      Join joinRel,
       RexNode condition) {
     double product =
         RelMetadataQuery.getRowCount(joinRel.getLeft())
@@ -219,9 +235,9 @@ public abstract class JoinRelBase extends AbstractRelNode {
   }
 
   /**
-   * Returns whether this JoinRel has already spawned a
-   * {@link org.eigenbase.rel.rules.SemiJoinRel} via
-   * {@link org.eigenbase.rel.rules.AddRedundantSemiJoinRule}.
+   * Returns whether this LogicalJoin has already spawned a
+   * {@link SemiJoin} via
+   * {@link org.apache.calcite.rel.rules.JoinAddRedundantSemiJoinRule}.
    *
    * <p>The base implementation returns false.</p>
    *
@@ -356,8 +372,7 @@ public abstract class JoinRelBase extends AbstractRelNode {
     }
   }
 
-  @Override
-  public final JoinRelBase copy(RelTraitSet traitSet, List<RelNode> inputs) {
+  @Override public final Join copy(RelTraitSet traitSet, List<RelNode> inputs) {
     assert inputs.size() == 2;
     return copy(traitSet, getCondition(), inputs.get(0), inputs.get(1),
         joinType, isSemiJoinDone());
@@ -367,7 +382,7 @@ public abstract class JoinRelBase extends AbstractRelNode {
    * Creates a copy of this join, overriding condition, system fields and
    * inputs.
    *
-   * <p>General contract as {@link org.eigenbase.rel.RelNode#copy}.
+   * <p>General contract as {@link RelNode#copy}.
    *
    * @param conditionExpr Condition
    * @param left          Left input
@@ -377,7 +392,7 @@ public abstract class JoinRelBase extends AbstractRelNode {
    *                      semi-join
    * @return Copy of this join
    */
-  public abstract JoinRelBase copy(RelTraitSet traitSet, RexNode conditionExpr,
+  public abstract Join copy(RelTraitSet traitSet, RexNode conditionExpr,
       RelNode left, RelNode right, JoinRelType joinType, boolean semiJoinDone);
 
   /** Analyzes the join condition. */
@@ -386,4 +401,4 @@ public abstract class JoinRelBase extends AbstractRelNode {
   }
 }
 
-// End JoinRelBase.java
+// End Join.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/core/JoinInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/JoinInfo.java b/core/src/main/java/org/apache/calcite/rel/core/JoinInfo.java
index f40374c..b1c8a18 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/JoinInfo.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/JoinInfo.java
@@ -14,32 +14,33 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel;
+package org.apache.calcite.rel.core;
+
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.ImmutableIntList;
+import org.apache.calcite.util.mapping.IntPair;
+
+import com.google.common.base.Preconditions;
 
 import java.util.ArrayList;
 import java.util.BitSet;
 import java.util.List;
 
-import org.eigenbase.relopt.RelOptUtil;
-import org.eigenbase.rex.RexBuilder;
-import org.eigenbase.rex.RexNode;
-import org.eigenbase.util.ImmutableIntList;
-import org.eigenbase.util.mapping.IntPair;
-
-import net.hydromatic.optiq.util.BitSets;
-
-import com.google.common.base.Preconditions;
-
 /** An analyzed join condition.
  *
  * <p>It is useful for the many algorithms that care whether a join is an
  * equi-join.
  *
- * <p>You can create one using {@link #of}, or call {@link JoinRelBase#analyzeCondition()};
- * many kinds of join cache their join info, especially those that are
- * equi-joins and sub-class {@link org.eigenbase.rel.rules.EquiJoinRel}.</p>
+ * <p>You can create one using {@link #of}, or call
+ * {@link Join#analyzeCondition()}; many kinds of join cache their
+ * join info, especially those that are equi-joins and sub-class
+ * {@link org.apache.calcite.rel.rules.EquiJoin}.</p>
  *
- * @see JoinRelBase#analyzeCondition() */
+ * @see Join#analyzeCondition() */
 public abstract class JoinInfo {
   public final ImmutableIntList leftKeys;
   public final ImmutableIntList rightKeys;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/core/JoinRelType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/JoinRelType.java b/core/src/main/java/org/apache/calcite/rel/core/JoinRelType.java
index 8fac256..677021e 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/JoinRelType.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/JoinRelType.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel;
+package org.apache.calcite.rel.core;
 
 /**
  * Enumeration of join types.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/core/Minus.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/Minus.java b/core/src/main/java/org/apache/calcite/rel/core/Minus.java
index 06039f8..12feba8 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/Minus.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/Minus.java
@@ -14,38 +14,42 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel;
+package org.apache.calcite.rel.core;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelInput;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.sql.SqlKind;
 
 import java.util.BitSet;
 import java.util.List;
 
-import org.eigenbase.rel.metadata.RelMetadataQuery;
-import org.eigenbase.relopt.RelOptCluster;
-import org.eigenbase.relopt.RelTraitSet;
-import org.eigenbase.sql.SqlKind;
-
 /**
- * Abstract base class for implementations of
- * {@link MinusRel}.
+ * Relational expression that returns the rows of its first input minus any
+ * matching rows from its other inputs.
+ *
+ * <p>Corresponds to the SQL {@code EXCEPT} operator.
+ *
+ * <p>If "all" is true, then multiset subtraction is
+ * performed; otherwise, set subtraction is performed (implying no duplicates in
+ * the results).
  */
-public abstract class MinusRelBase extends SetOpRel {
-  public MinusRelBase(
-      RelOptCluster cluster,
-      RelTraitSet traits,
-      List<RelNode> inputs,
+public abstract class Minus extends SetOp {
+  public Minus(RelOptCluster cluster, RelTraitSet traits, List<RelNode> inputs,
       boolean all) {
     super(cluster, traits, inputs, SqlKind.EXCEPT, all);
   }
 
   /**
-   * Creates a MinusRelBase by parsing serialized output.
+   * Creates a Minus by parsing serialized output.
    */
-  protected MinusRelBase(RelInput input) {
+  protected Minus(RelInput input) {
     super(input);
   }
 
-  @Override
-  public double getRows() {
+  @Override public double getRows() {
     // REVIEW jvs 30-May-2005:  I just pulled this out of a hat.
     double dRows = RelMetadataQuery.getRowCount(inputs.get(0));
     for (int i = 1; i < inputs.size(); i++) {
@@ -57,11 +61,10 @@ public abstract class MinusRelBase extends SetOpRel {
     return dRows;
   }
 
-  @Override
-  public boolean isKey(BitSet columns) {
+  @Override public boolean isKey(BitSet columns) {
     return inputs.get(0).isKey(columns)
         || super.isKey(columns);
   }
 }
 
-// End MinusRelBase.java
+// End Minus.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/core/OneRow.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/OneRow.java b/core/src/main/java/org/apache/calcite/rel/core/OneRow.java
index 5ae37e6..a3ccc5b 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/OneRow.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/OneRow.java
@@ -14,34 +14,42 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel;
+package org.apache.calcite.rel.core;
 
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.type.*;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.AbstractRelNode;
+import org.apache.calcite.rel.RelInput;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.sql.type.SqlTypeName;
 
 /**
- * <code>OneRowRelBase</code> is an abstract base class for implementations of
- * {@link OneRowRel}.
+ * Relational expression that always returns one row.
+ *
+ * <p>It has one column, called "ZERO", containing the value 0.
+ *
+ * @see Values
  */
-public abstract class OneRowRelBase extends AbstractRelNode {
+public abstract class OneRow extends AbstractRelNode {
   //~ Constructors -----------------------------------------------------------
 
   /**
-   * Creates a <code>OneRowRelBase</code> with specific traits.
+   * Creates a <code>OneRow</code>.
    *
-   * @param cluster {@link RelOptCluster}  this relational expression belongs
-   *                to
-   * @param traits  for this rel
+   * @param cluster   Cluster that this relational expression belongs to
+   * @param traits    Traits
    */
-  protected OneRowRelBase(RelOptCluster cluster, RelTraitSet traits) {
+  protected OneRow(RelOptCluster cluster, RelTraitSet traits) {
     super(cluster, traits);
   }
 
   /**
-   * Creates a OneRowRelBase by parsing serialized output.
+   * Creates a OneRow by parsing serialized output.
    */
-  protected OneRowRelBase(RelInput input) {
+  protected OneRow(RelInput input) {
     this(input.getCluster(), input.getTraitSet());
   }
 
@@ -60,4 +68,4 @@ public abstract class OneRowRelBase extends AbstractRelNode {
   }
 }
 
-// End OneRowRelBase.java
+// End OneRow.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 68c40f9..7ff2cc4 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
@@ -14,32 +14,54 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel;
-
-import java.util.*;
-
-import org.eigenbase.rel.metadata.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.rex.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.util.Pair;
-import org.eigenbase.util.Permutation;
-import org.eigenbase.util.Util;
-import org.eigenbase.util.mapping.MappingType;
-import org.eigenbase.util.mapping.Mappings;
-
-import net.hydromatic.linq4j.Ord;
-import net.hydromatic.linq4j.function.Function1;
-import net.hydromatic.linq4j.function.Functions;
+package org.apache.calcite.rel.core;
+
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.linq4j.function.Function1;
+import org.apache.calcite.linq4j.function.Functions;
+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.RelCollation;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.RelInput;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelWriter;
+import org.apache.calcite.rel.SingleRel;
+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.RexUtil;
+import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Permutation;
+import org.apache.calcite.util.Util;
+import org.apache.calcite.util.mapping.MappingType;
+import org.apache.calcite.util.mapping.Mappings;
 
 import com.google.common.collect.ImmutableList;
 
+import java.util.List;
+
 /**
- * <code>ProjectRelBase</code> is an abstract base class for implementations of
- * {@link ProjectRel}.
+ * Relational expression that computes a set of
+ * 'select expressions' from its input relational expression.
+ *
+ * <p>The result is usually 'boxed' as a record with one named field for each
+ * column; if there is precisely one expression, the result may be 'unboxed',
+ * and consist of the raw value type.
+ *
+ * @see org.apache.calcite.rel.logical.LogicalProject
  */
-public abstract class ProjectRelBase extends SingleRel {
+public abstract class Project extends SingleRel {
   //~ Instance fields --------------------------------------------------------
 
   protected final ImmutableList<RexNode> exps;
@@ -56,21 +78,22 @@ public abstract class ProjectRelBase extends SingleRel {
   /**
    * Creates a Project.
    *
-   * @param cluster Cluster this relational expression belongs to
+   * @param cluster Cluster that this relational expression belongs to
    * @param traits  traits of this rel
-   * @param child   input relational expression
+   * @param input   input relational expression
    * @param exps    List of expressions for the input columns
    * @param rowType output row type
-   * @param flags   values as in {@link Flags}
+   * @param flags      Flags; values as in {@link Project.Flags},
+   *                   usually {@link Project.Flags#BOXED}
    */
-  protected ProjectRelBase(
+  protected Project(
       RelOptCluster cluster,
       RelTraitSet traits,
-      RelNode child,
+      RelNode input,
       List<? extends RexNode> exps,
       RelDataType rowType,
       int flags) {
-    super(cluster, traits, child);
+    super(cluster, traits, input);
     assert rowType != null;
     this.exps = ImmutableList.copyOf(exps);
     this.rowType = rowType;
@@ -85,11 +108,10 @@ public abstract class ProjectRelBase extends SingleRel {
   }
 
   /**
-   * Creates a ProjectRelBase by parsing serialized output.
+   * Creates a Project by parsing serialized output.
    */
-  protected ProjectRelBase(RelInput input) {
-    this(
-        input.getCluster(), input.getTraitSet(), input.getInput(),
+  protected Project(RelInput input) {
+    this(input.getCluster(), input.getTraitSet(), input.getInput(),
         input.getExpressionList("exprs"),
         input.getRowType("exprs", "fields"), Flags.BOXED);
   }
@@ -104,7 +126,7 @@ public abstract class ProjectRelBase extends SingleRel {
   /** Copies a project.
    *
    * @see #copy(RelTraitSet, List) */
-  public abstract ProjectRelBase copy(RelTraitSet traitSet, RelNode input,
+  public abstract Project copy(RelTraitSet traitSet, RelNode input,
       List<RexNode> exps, RelDataType rowType);
 
   public List<RelCollation> getCollationList() {
@@ -115,8 +137,7 @@ public abstract class ProjectRelBase extends SingleRel {
     return (flags & Flags.BOXED) == Flags.BOXED;
   }
 
-  @Override
-  public List<RexNode> getChildExps() {
+  @Override public List<RexNode> getChildExps() {
     return exps;
   }
 
@@ -153,7 +174,7 @@ public abstract class ProjectRelBase extends SingleRel {
     }
     RexChecker checker =
         new RexChecker(
-            getChild().getRowType(), fail);
+            getInput().getRowType(), fail);
     for (RexNode exp : exps) {
       exp.accept(checker);
     }
@@ -203,7 +224,7 @@ public abstract class ProjectRelBase extends SingleRel {
   }
 
   public RelOptCost computeSelfCost(RelOptPlanner planner) {
-    double dRows = RelMetadataQuery.getRowCount(getChild());
+    double dRows = RelMetadataQuery.getRowCount(getInput());
     double dCpu = dRows * exps.size();
     double dIo = 0;
     return planner.getCostFactory().makeCost(dRows, dCpu, dIo);
@@ -241,7 +262,7 @@ public abstract class ProjectRelBase extends SingleRel {
    * Returns a mapping, or null if this projection is not a mapping.
    */
   public Mappings.TargetMapping getMapping() {
-    return getMapping(getChild().getRowType().getFieldCount(), exps);
+    return getMapping(getInput().getRowType().getFieldCount(), exps);
   }
 
   /**
@@ -276,7 +297,7 @@ public abstract class ProjectRelBase extends SingleRel {
    */
   public Permutation getPermutation() {
     final int fieldCount = rowType.getFieldList().size();
-    if (fieldCount != getChild().getRowType().getFieldList().size()) {
+    if (fieldCount != getInput().getRowType().getFieldList().size()) {
       return null;
     }
     Permutation permutation = new Permutation(fieldCount);
@@ -398,4 +419,4 @@ public abstract class ProjectRelBase extends SingleRel {
   }
 }
 
-// End ProjectRelBase.java
+// End Project.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/core/RelFactories.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/RelFactories.java b/core/src/main/java/org/apache/calcite/rel/core/RelFactories.java
index b5d9634..fcc9a2d 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/RelFactories.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/RelFactories.java
@@ -15,22 +15,29 @@
  * limitations under the License.
  */
 
-package org.eigenbase.rel;
+package org.apache.calcite.rel.core;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.logical.LogicalAggregate;
+import org.apache.calcite.rel.logical.LogicalFilter;
+import org.apache.calcite.rel.logical.LogicalIntersect;
+import org.apache.calcite.rel.logical.LogicalJoin;
+import org.apache.calcite.rel.logical.LogicalMinus;
+import org.apache.calcite.rel.logical.LogicalUnion;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.SqlKind;
+
+import com.google.common.collect.ImmutableList;
 
 import java.util.BitSet;
 import java.util.List;
 import java.util.Set;
 
-import org.eigenbase.rel.rules.SemiJoinRel;
-import org.eigenbase.relopt.RelOptCluster;
-import org.eigenbase.relopt.RelOptUtil;
-import org.eigenbase.relopt.RelTraitSet;
-import org.eigenbase.reltype.RelDataTypeField;
-import org.eigenbase.rex.RexNode;
-import org.eigenbase.sql.SqlKind;
-
-import com.google.common.collect.ImmutableList;
-
 /**
  * Contains factory interface and default implementation for creating various
  * rel nodes.
@@ -60,8 +67,9 @@ public class RelFactories {
   }
 
   /**
-   * Can create a {@link org.eigenbase.rel.ProjectRel} of the appropriate type
-   * for this rule's calling convention.
+   * Can create a
+   * {@link org.apache.calcite.rel.logical.LogicalProject} of the
+   * appropriate type for this rule's calling convention.
    */
   public interface ProjectFactory {
     /** Creates a project. */
@@ -71,7 +79,7 @@ public class RelFactories {
 
   /**
    * Implementation of {@link ProjectFactory} that returns a vanilla
-   * {@link ProjectRel}.
+   * {@link org.apache.calcite.rel.logical.LogicalProject}.
    */
   private static class ProjectFactoryImpl implements ProjectFactory {
     public RelNode createProject(RelNode child,
@@ -81,7 +89,7 @@ public class RelFactories {
   }
 
   /**
-   * Can create a {@link org.eigenbase.rel.SortRel} of the appropriate type
+   * Can create a {@link Sort} of the appropriate type
    * for this rule's calling convention.
    */
   public interface SortFactory {
@@ -91,19 +99,19 @@ public class RelFactories {
   }
 
   /**
-   * Implementation of {@link org.eigenbase.rel.RelFactories.SortFactory} that
-   * returns a vanilla {@link SortRel}.
+   * Implementation of {@link RelFactories.SortFactory} that
+   * returns a vanilla {@link Sort}.
    */
   private static class SortFactoryImpl implements SortFactory {
     public RelNode createSort(RelTraitSet traits, RelNode child,
         RelCollation collation, RexNode offset, RexNode fetch) {
-      return new SortRel(child.getCluster(), traits, child, collation,
+      return new Sort(child.getCluster(), traits, child, collation,
           offset, fetch);
     }
   }
 
   /**
-   * Can create a {@link org.eigenbase.rel.SetOpRel} for a particular kind of
+   * Can create a {@link SetOp} for a particular kind of
    * set operation (UNION, EXCEPT, INTERSECT) and of the appropriate type
    * for this rule's calling convention.
    */
@@ -113,8 +121,8 @@ public class RelFactories {
   }
 
   /**
-   * Implementation of {@link org.eigenbase.rel.RelFactories.SetOpFactory} that
-   * returns a vanilla {@link SetOpRel} for the particular kind of set
+   * Implementation of {@link RelFactories.SetOpFactory} that
+   * returns a vanilla {@link SetOp} for the particular kind of set
    * operation (UNION, EXCEPT, INTERSECT).
    */
   private static class SetOpFactoryImpl implements SetOpFactory {
@@ -123,11 +131,11 @@ public class RelFactories {
       final RelOptCluster cluster = inputs.get(0).getCluster();
       switch (kind) {
       case UNION:
-        return new UnionRel(cluster, inputs, all);
+        return new LogicalUnion(cluster, inputs, all);
       case EXCEPT:
-        return new MinusRel(cluster, inputs, all);
+        return new LogicalMinus(cluster, inputs, all);
       case INTERSECT:
-        return new IntersectRel(cluster, inputs, all);
+        return new LogicalIntersect(cluster, inputs, all);
       default:
         throw new AssertionError("not a set op: " + kind);
       }
@@ -135,7 +143,7 @@ public class RelFactories {
   }
 
   /**
-   * Can create a {@link org.eigenbase.rel.AggregateRel} of the appropriate type
+   * Can create a {@link LogicalAggregate} of the appropriate type
    * for this rule's calling convention.
    */
   public interface AggregateFactory {
@@ -145,18 +153,19 @@ public class RelFactories {
   }
 
   /**
-   * Implementation of {@link org.eigenbase.rel.RelFactories.AggregateFactory}
-   * that returns a vanilla {@link AggregateRel}.
+   * Implementation of {@link RelFactories.AggregateFactory}
+   * that returns a vanilla {@link LogicalAggregate}.
    */
   private static class AggregateFactoryImpl implements AggregateFactory {
     public RelNode createAggregate(RelNode child, BitSet groupSet,
         List<AggregateCall> aggCalls) {
-      return new AggregateRel(child.getCluster(), child, groupSet, aggCalls);
+      return new LogicalAggregate(child.getCluster(), child, groupSet,
+          aggCalls);
     }
   }
 
   /**
-   * Can create a {@link org.eigenbase.rel.FilterRel} of the appropriate type
+   * Can create a {@link LogicalFilter} of the appropriate type
    * for this rule's calling convention.
    */
   public interface FilterFactory {
@@ -165,19 +174,19 @@ public class RelFactories {
   }
 
   /**
-   * Implementation of {@link org.eigenbase.rel.RelFactories.FilterFactory} that
-   * returns a vanilla {@link FilterRel}.
+   * Implementation of {@link RelFactories.FilterFactory} that
+   * returns a vanilla {@link LogicalFilter}.
    */
   private static class FilterFactoryImpl implements FilterFactory {
     public RelNode createFilter(RelNode child, RexNode condition) {
-      return new FilterRel(child.getCluster(), child, condition);
+      return new LogicalFilter(child.getCluster(), child, condition);
     }
   }
 
   /**
    * Can create a join of the appropriate type for a rule's calling convention.
    *
-   * <p>The result is typically a {@link org.eigenbase.rel.JoinRelBase}.
+   * <p>The result is typically a {@link Join}.
    */
   public interface JoinFactory {
     /**
@@ -189,7 +198,7 @@ public class RelFactories {
      * @param joinType         Join type
      * @param variablesStopped Set of names of variables which are set by the
      *                         LHS and used by the RHS and are not available to
-     *                         nodes above this JoinRel in the tree
+     *                         nodes above this LogicalJoin in the tree
      * @param semiJoinDone     Whether this join has been translated to a
      *                         semi-join
      */
@@ -200,14 +209,14 @@ public class RelFactories {
 
   /**
    * Implementation of {@link JoinFactory} that returns a vanilla
-   * {@link JoinRel}.
+   * {@link org.apache.calcite.rel.logical.LogicalJoin}.
    */
   private static class JoinFactoryImpl implements JoinFactory {
     public RelNode createJoin(RelNode left, RelNode right,
         RexNode condition, JoinRelType joinType,
         Set<String> variablesStopped, boolean semiJoinDone) {
       final RelOptCluster cluster = left.getCluster();
-      return new JoinRel(cluster, left, right, condition, joinType,
+      return new LogicalJoin(cluster, left, right, condition, joinType,
           variablesStopped, semiJoinDone, ImmutableList.<RelDataTypeField>of());
     }
   }
@@ -229,13 +238,13 @@ public class RelFactories {
 
   /**
    * Implementation of {@link SemiJoinFactory} that returns a vanilla
-   * {@link SemiJoinRel}.
+   * {@link SemiJoin}.
    */
   private static class SemiJoinFactoryImpl implements SemiJoinFactory {
     public RelNode createSemiJoin(RelNode left, RelNode right,
         RexNode condition) {
       final JoinInfo joinInfo = JoinInfo.of(left, right, condition);
-      return new SemiJoinRel(left.getCluster(), left.getTraitSet(), left, right,
+      return new SemiJoin(left.getCluster(), left.getTraitSet(), left, right,
         condition, joinInfo.leftKeys, joinInfo.rightKeys);
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/core/Sample.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/Sample.java b/core/src/main/java/org/apache/calcite/rel/core/Sample.java
index ee22849..e22e0de 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/Sample.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/Sample.java
@@ -14,40 +14,43 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel;
+package org.apache.calcite.rel.core;
 
-import java.util.List;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptSamplingParameters;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelInput;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelWriter;
+import org.apache.calcite.rel.SingleRel;
 
-import org.eigenbase.relopt.*;
+import java.util.List;
 
 /**
- * SamplingRel represents the TABLESAMPLE BERNOULLI or SYSTEM keyword applied to
- * a table, view or subquery.
+ * Relational expression that returns a sample of the rows from its input.
+ *
+ * <p>In SQL, a sample is expressed using the {@code TABLESAMPLE BERNOULLI} or
+ * {@code SYSTEM} keyword applied to a table, view or subquery.
  */
-public class SamplingRel extends SingleRel {
+public class Sample extends SingleRel {
   //~ Instance fields --------------------------------------------------------
 
   private final RelOptSamplingParameters params;
 
   //~ Constructors -----------------------------------------------------------
 
-  public SamplingRel(
-      RelOptCluster cluster,
-      RelNode child,
+  public Sample(RelOptCluster cluster, RelNode child,
       RelOptSamplingParameters params) {
-    super(
-        cluster,
-        cluster.traitSetOf(Convention.NONE),
-        child);
+    super(cluster, cluster.traitSetOf(Convention.NONE), child);
     this.params = params;
   }
 
   /**
-   * Creates a SamplingRel by parsing serialized output.
+   * Creates a Sample by parsing serialized output.
    */
-  public SamplingRel(RelInput input) {
-    this(
-        input.getCluster(), input.getInput(), getSamplingParameters(input));
+  public Sample(RelInput input) {
+    this(input.getCluster(), input.getInput(), getSamplingParameters(input));
   }
 
   //~ Methods ----------------------------------------------------------------
@@ -65,28 +68,23 @@ public class SamplingRel extends SingleRel {
 
   public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
     assert traitSet.containsIfApplicable(Convention.NONE);
-    return new SamplingRel(
-        getCluster(),
-        sole(inputs),
-        params);
+    return new Sample(getCluster(), sole(inputs), params);
   }
 
   /**
-   * Retrieve the sampling parameters for this SamplingRel.
+   * Retrieve the sampling parameters for this Sample.
    */
   public RelOptSamplingParameters getSamplingParameters() {
     return params;
   }
 
-  // implement RelNode
-  public RelWriter explainTerms(RelWriter pw) {
+  @Override public RelWriter explainTerms(RelWriter pw) {
     return super.explainTerms(pw)
         .item("mode", params.isBernoulli() ? "bernoulli" : "system")
         .item("rate", params.getSamplingPercentage())
-        .item(
-            "repeatableSeed",
+        .item("repeatableSeed",
             params.isRepeatable() ? params.getRepeatableSeed() : "-");
   }
 }
 
-// End SamplingRel.java
+// End Sample.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/core/SemiJoin.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/SemiJoin.java b/core/src/main/java/org/apache/calcite/rel/core/SemiJoin.java
index c78c343..17347bf 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/SemiJoin.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/SemiJoin.java
@@ -14,39 +14,46 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
+package org.apache.calcite.rel.core;
 
-import java.util.*;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.rel.metadata.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.rex.*;
-import org.eigenbase.util.ImmutableIntList;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rel.rules.EquiJoin;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.util.ImmutableIntList;
 
+import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
 
 /**
- * A SemiJoinRel represents two relational expressions joined according to some
- * condition, where the output only contains the columns from the left join
- * input.
+ * Relational expression that joins two relational expressions according to some
+ * condition, but outputs only columns from the left input, and eliminates
+ * duplicates.
+ *
+ * <p>The effect is something like the SQL {@code IN} operator.
  */
-public class SemiJoinRel extends EquiJoinRel {
+public class SemiJoin extends EquiJoin {
   //~ Constructors -----------------------------------------------------------
 
   /**
-   * Creates a SemiJoinRel.
+   * Creates a SemiJoin.
    *
    * @param cluster   cluster that join belongs to
-   * @param traitSet  Traits
+   * @param traitSet  Trait set
    * @param left      left join input
    * @param right     right join input
    * @param condition join condition
    * @param leftKeys  left keys of the semijoin
    * @param rightKeys right keys of the semijoin
    */
-  public SemiJoinRel(
+  public SemiJoin(
       RelOptCluster cluster,
       RelTraitSet traitSet,
       RelNode left,
@@ -68,41 +75,41 @@ public class SemiJoinRel extends EquiJoinRel {
 
   //~ Methods ----------------------------------------------------------------
 
-  @Override
-  public SemiJoinRel copy(RelTraitSet traitSet, RexNode condition,
+  @Override public SemiJoin copy(RelTraitSet traitSet, RexNode condition,
       RelNode left, RelNode right, JoinRelType joinType, boolean semiJoinDone) {
     assert joinType == JoinRelType.INNER;
     final JoinInfo joinInfo = JoinInfo.of(left, right, condition);
     assert joinInfo.isEqui();
-    return new SemiJoinRel(getCluster(), traitSet, left, right, condition,
+    return new SemiJoin(getCluster(), traitSet, left, right, condition,
         joinInfo.leftKeys, joinInfo.rightKeys);
   }
 
-  // implement RelNode
-  public RelOptCost computeSelfCost(RelOptPlanner planner) {
+  @Override public RelOptCost computeSelfCost(RelOptPlanner planner) {
     // REVIEW jvs 9-Apr-2006:  Just for now...
     return planner.getCostFactory().makeTinyCost();
   }
 
-  // implement RelNode
-  public double getRows() {
+  @Override public double getRows() {
     // TODO:  correlation factor
     return RelMetadataQuery.getRowCount(left)
         * RexUtil.getSelectivity(condition);
   }
 
   /**
-   * @return returns rowtype representing only the left join input
+   * {@inheritDoc}
+   *
+   * <p>In the case of semi-join, the row type consists of columns from left
+   * input only.
    */
-  public RelDataType deriveRowType() {
+  @Override public RelDataType deriveRowType() {
     return deriveJoinRowType(
         left.getRowType(),
         null,
         JoinRelType.INNER,
         getCluster().getTypeFactory(),
         null,
-        Collections.<RelDataTypeField>emptyList());
+        ImmutableList.<RelDataTypeField>of());
   }
 }
 
-// End SemiJoinRel.java
+// End SemiJoin.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/core/SetOp.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/SetOp.java b/core/src/main/java/org/apache/calcite/rel/core/SetOp.java
index 6a650e4..7b0187a 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/SetOp.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/SetOp.java
@@ -14,24 +14,32 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel;
-
-import java.util.*;
-
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.SqlKind;
-
-import net.hydromatic.linq4j.Ord;
+package org.apache.calcite.rel.core;
+
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.AbstractRelNode;
+import org.apache.calcite.rel.RelInput;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelWriter;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlKind;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 
+import java.util.AbstractList;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.List;
+
 /**
- * <code>SetOpRel</code> is an abstract base for relational set operators such
+ * <code>SetOp</code> is an abstract base for relational set operators such
  * as UNION, MINUS (aka EXCEPT), and INTERSECT.
  */
-public abstract class SetOpRel extends AbstractRelNode {
+public abstract class SetOp extends AbstractRelNode {
   //~ Instance fields --------------------------------------------------------
 
   protected ImmutableList<RelNode> inputs;
@@ -41,14 +49,10 @@ public abstract class SetOpRel extends AbstractRelNode {
   //~ Constructors -----------------------------------------------------------
 
   /**
-   * Creates a SetOpRel.
+   * Creates a SetOp.
    */
-  protected SetOpRel(
-      RelOptCluster cluster,
-      RelTraitSet traits,
-      List<RelNode> inputs,
-      SqlKind kind,
-      boolean all) {
+  protected SetOp(RelOptCluster cluster, RelTraitSet traits,
+      List<RelNode> inputs, SqlKind kind, boolean all) {
     super(cluster, traits);
     Preconditions.checkArgument(kind == SqlKind.UNION
         || kind == SqlKind.INTERSECT
@@ -59,49 +63,42 @@ public abstract class SetOpRel extends AbstractRelNode {
   }
 
   /**
-   * Creates a SetOpRel by parsing serialized output.
+   * Creates a SetOp by parsing serialized output.
    */
-  protected SetOpRel(RelInput input) {
+  protected SetOp(RelInput input) {
     this(input.getCluster(), input.getTraitSet(), input.getInputs(),
         SqlKind.UNION, input.getBoolean("all"));
   }
 
   //~ Methods ----------------------------------------------------------------
 
-  public abstract SetOpRel copy(
+  public abstract SetOp copy(
       RelTraitSet traitSet,
       List<RelNode> inputs,
       boolean all);
 
-  @Override
-  public SetOpRel copy(
-      RelTraitSet traitSet,
-      List<RelNode> inputs) {
+  @Override public SetOp copy(RelTraitSet traitSet, List<RelNode> inputs) {
     return copy(traitSet, inputs, all);
   }
 
-  @Override
-  public void replaceInput(int ordinalInParent, RelNode p) {
+  @Override public void replaceInput(int ordinalInParent, RelNode p) {
     final List<RelNode> newInputs = new ArrayList<RelNode>(inputs);
     newInputs.set(ordinalInParent, p);
     inputs = ImmutableList.copyOf(newInputs);
     recomputeDigest();
   }
 
-  @Override
-  public boolean isKey(BitSet columns) {
+  @Override public boolean isKey(BitSet columns) {
     // If not ALL then the rows are distinct.
     // Therefore the set of all columns is a key.
     return !all && columns.nextClearBit(0) >= getRowType().getFieldCount();
   }
 
-  @Override
-  public List<RelNode> getInputs() {
+  @Override public List<RelNode> getInputs() {
     return inputs;
   }
 
-  @Override
-  public RelWriter explainTerms(RelWriter pw) {
+  @Override public RelWriter explainTerms(RelWriter pw) {
     super.explainTerms(pw);
     for (Ord<RelNode> ord : Ord.zip(inputs)) {
       pw.input("input#" + ord.i, ord.e);
@@ -109,17 +106,14 @@ public abstract class SetOpRel extends AbstractRelNode {
     return pw.item("all", all);
   }
 
-  @Override
-  protected RelDataType deriveRowType() {
+  @Override protected RelDataType deriveRowType() {
     return getCluster().getTypeFactory().leastRestrictive(
         new AbstractList<RelDataType>() {
-          @Override
-          public RelDataType get(int index) {
+          @Override public RelDataType get(int index) {
             return inputs.get(index).getRowType();
           }
 
-          @Override
-          public int size() {
+          @Override public int size() {
             return inputs.size();
           }
         });
@@ -129,7 +123,7 @@ public abstract class SetOpRel extends AbstractRelNode {
    * Returns whether all the inputs of this set operator have the same row
    * type as its output row.
    *
-   * @param compareNames whether or not column names are important in the
+   * @param compareNames Whether column names are important in the
    *                     homogeneity comparison
    */
   public boolean isHomogeneous(boolean compareNames) {
@@ -142,15 +136,6 @@ public abstract class SetOpRel extends AbstractRelNode {
     }
     return true;
   }
-
-  /**
-   * Returns whether all the inputs of this set operator have the same row
-   * type as its output row. Equivalent to {@link #isHomogeneous(boolean)
-   * isHomogeneous(true)}.
-   */
-  public boolean isHomogeneous() {
-    return isHomogeneous(true);
-  }
 }
 
-// End SetOpRel.java
+// End SetOp.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/core/Sort.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/Sort.java b/core/src/main/java/org/apache/calcite/rel/core/Sort.java
index 11ab22b..5c91666 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/Sort.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/Sort.java
@@ -14,25 +14,36 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel;
+package org.apache.calcite.rel.core;
+
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.RelInput;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelShuttle;
+import org.apache.calcite.rel.RelWriter;
+import org.apache.calcite.rel.SingleRel;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.util.Util;
+
+import com.google.common.collect.ImmutableList;
 
 import java.util.Collections;
 import java.util.List;
 
-import org.eigenbase.rel.metadata.RelMetadataQuery;
-import org.eigenbase.relopt.*;
-import org.eigenbase.rex.*;
-import org.eigenbase.util.Util;
-
-import net.hydromatic.linq4j.Ord;
-
-import com.google.common.collect.ImmutableList;
-
 /**
- * Relational expression which imposes a particular sort order on its input
+ * Relational expression that imposes a particular sort order on its input
  * without otherwise changing its content.
  */
-public class SortRel extends SingleRel {
+public class Sort extends SingleRel {
   //~ Instance fields --------------------------------------------------------
 
   protected final RelCollation collation;
@@ -43,14 +54,14 @@ public class SortRel extends SingleRel {
   //~ Constructors -----------------------------------------------------------
 
   /**
-   * Creates a sorter.
+   * Creates a Sort.
    *
    * @param cluster   Cluster this relational expression belongs to
    * @param traits    Traits
    * @param child     input relational expression
    * @param collation array of sort specifications
    */
-  public SortRel(
+  public Sort(
       RelOptCluster cluster,
       RelTraitSet traits,
       RelNode child,
@@ -59,7 +70,7 @@ public class SortRel extends SingleRel {
   }
 
   /**
-   * Creates a sorter.
+   * Creates a Sort.
    *
    * @param cluster   Cluster this relational expression belongs to
    * @param traits    Traits
@@ -69,7 +80,7 @@ public class SortRel extends SingleRel {
    *                  first row
    * @param fetch     Expression for number of rows to fetch
    */
-  public SortRel(
+  public Sort(
       RelOptCluster cluster,
       RelTraitSet traits,
       RelNode child,
@@ -96,11 +107,10 @@ public class SortRel extends SingleRel {
   }
 
   /**
-   * Creates a SortRel by parsing serialized output.
+   * Creates a Sort by parsing serialized output.
    */
-  public SortRel(RelInput input) {
-    this(
-        input.getCluster(), input.getTraitSet().plus(input.getCollation()),
+  public Sort(RelInput input) {
+    this(input.getCluster(), input.getTraitSet().plus(input.getCollation()),
         input.getInput(),
         RelCollationTraitDef.INSTANCE.canonize(input.getCollation()),
         input.getExpression("offset"), input.getExpression("fetch"));
@@ -108,26 +118,25 @@ public class SortRel extends SingleRel {
 
   //~ Methods ----------------------------------------------------------------
 
-  @Override
-  public SortRel copy(RelTraitSet traitSet, List<RelNode> inputs) {
+  @Override public Sort copy(RelTraitSet traitSet, List<RelNode> inputs) {
     return copy(traitSet, sole(inputs), collation);
   }
 
-  public SortRel copy(
+  public Sort copy(
       RelTraitSet traitSet,
       RelNode newInput,
       RelCollation newCollation) {
     return copy(traitSet, newInput, newCollation, offset, fetch);
   }
 
-  public SortRel copy(
+  public Sort copy(
       RelTraitSet traitSet,
       RelNode newInput,
       RelCollation newCollation,
       RexNode offset,
       RexNode fetch) {
     assert traitSet.containsIfApplicable(Convention.NONE);
-    return new SortRel(
+    return new Sort(
         getCluster(),
         traitSet,
         newInput,
@@ -136,8 +145,7 @@ public class SortRel extends SingleRel {
         fetch);
   }
 
-  @Override
-  public RelOptCost computeSelfCost(RelOptPlanner planner) {
+  @Override public RelOptCost computeSelfCost(RelOptPlanner planner) {
     // Higher cost if rows are wider discourages pushing a project through a
     // sort.
     double rowCount = RelMetadataQuery.getRowCount(this);
@@ -146,13 +154,11 @@ public class SortRel extends SingleRel {
         Util.nLogN(rowCount) * bytesPerRow, rowCount, 0);
   }
 
-  @Override
-  public RelNode accept(RelShuttle shuttle) {
+  @Override public RelNode accept(RelShuttle shuttle) {
     return shuttle.visit(this);
   }
 
-  @Override
-  public List<RexNode> getChildExps() {
+  @Override public List<RexNode> getChildExps() {
     return fieldExps;
   }
 
@@ -172,8 +178,7 @@ public class SortRel extends SingleRel {
     return collation;
   }
 
-  @Override
-  public List<RelCollation> getCollationList() {
+  @Override public List<RelCollation> getCollationList() {
     // TODO: include each prefix of the collation, e.g [[x, y], [x], []]
     return Collections.singletonList(getCollation());
   }
@@ -198,4 +203,4 @@ public class SortRel extends SingleRel {
   }
 }
 
-// End SortRel.java
+// End Sort.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/core/TableFunctionScan.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/TableFunctionScan.java b/core/src/main/java/org/apache/calcite/rel/core/TableFunctionScan.java
index 9a31fc4..282b1f3 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/TableFunctionScan.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/TableFunctionScan.java
@@ -14,26 +14,38 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel;
-
-import java.lang.reflect.Type;
-import java.util.*;
-
-import org.eigenbase.rel.metadata.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.rex.*;
-
-import net.hydromatic.linq4j.Ord;
+package org.apache.calcite.rel.core;
+
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.AbstractRelNode;
+import org.apache.calcite.rel.RelInput;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelWriter;
+import org.apache.calcite.rel.metadata.RelColumnMapping;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexNode;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
 
+import java.lang.reflect.Type;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+
 /**
- * <code>TableFunctionRelBase</code> is an abstract base class for
- * implementations of {@link TableFunctionRel}.
+ * Relational expression that calls a table-valued function.
+ *
+ * <p>The function returns a result set.
+ * It can appear as a leaf in a query tree,
+ * or can be applied to relational inputs.
+ *
+ * @see org.apache.calcite.rel.logical.LogicalTableFunctionScan
  */
-public abstract class TableFunctionRelBase extends AbstractRelNode {
+public abstract class TableFunctionScan extends AbstractRelNode {
   //~ Instance fields --------------------------------------------------------
 
   private final RexNode rexCall;
@@ -47,7 +59,7 @@ public abstract class TableFunctionRelBase extends AbstractRelNode {
   //~ Constructors -----------------------------------------------------------
 
   /**
-   * Creates a <code>TableFunctionRelBase</code>.
+   * Creates a <code>TableFunctionScan</code>.
    *
    * @param cluster        Cluster that this relational expression belongs to
    * @param inputs         0 or more relational inputs
@@ -57,7 +69,7 @@ public abstract class TableFunctionRelBase extends AbstractRelNode {
    * @param rowType        row type produced by function
    * @param columnMappings column mappings associated with this function
    */
-  protected TableFunctionRelBase(
+  protected TableFunctionScan(
       RelOptCluster cluster,
       RelTraitSet traits,
       List<RelNode> inputs,
@@ -74,9 +86,9 @@ public abstract class TableFunctionRelBase extends AbstractRelNode {
   }
 
   /**
-   * Creates a TableFunctionRelBase by parsing serialized output.
+   * Creates a TableFunctionScan by parsing serialized output.
    */
-  protected TableFunctionRelBase(RelInput input) {
+  protected TableFunctionScan(RelInput input) {
     this(
         input.getCluster(), input.getTraitSet(), input.getInputs(),
         input.getExpression("invocation"), (Type) input.get("elementType"),
@@ -86,26 +98,22 @@ public abstract class TableFunctionRelBase extends AbstractRelNode {
 
   //~ Methods ----------------------------------------------------------------
 
-  @Override
-  public List<RelNode> getInputs() {
+  @Override public List<RelNode> getInputs() {
     return inputs;
   }
 
-  @Override
-  public List<RexNode> getChildExps() {
+  @Override public List<RexNode> getChildExps() {
     return ImmutableList.of(rexCall);
   }
 
-  @Override
-  public void replaceInput(int ordinalInParent, RelNode p) {
+  @Override public void replaceInput(int ordinalInParent, RelNode p) {
     final List<RelNode> newInputs = new ArrayList<RelNode>(inputs);
     newInputs.set(ordinalInParent, p);
     inputs = ImmutableList.copyOf(newInputs);
     recomputeDigest();
   }
 
-  @Override
-  public double getRows() {
+  @Override public double getRows() {
     // Calculate result as the sum of the input rowcount estimates,
     // assuming there are any, otherwise use the superclass default.  So
     // for a no-input UDX, behave like an AbstractRelNode; for a one-input
@@ -161,4 +169,4 @@ public abstract class TableFunctionRelBase extends AbstractRelNode {
   }
 }
 
-// End TableFunctionRelBase.java
+// End TableFunctionScan.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/core/TableModify.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/TableModify.java b/core/src/main/java/org/apache/calcite/rel/core/TableModify.java
index 0f6dc26..fea3a5e 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/TableModify.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/TableModify.java
@@ -14,23 +14,40 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel;
-
-import java.util.*;
-
-import org.eigenbase.rel.metadata.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.SqlKind;
-import org.eigenbase.sql.type.*;
-
-import net.hydromatic.optiq.prepare.Prepare;
+package org.apache.calcite.rel.core;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.prepare.Prepare;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelWriter;
+import org.apache.calcite.rel.SingleRel;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.type.SqlTypeUtil;
+
+import java.util.Collections;
+import java.util.List;
 
 /**
- * <code>TableModificationRelBase</code> is an abstract base class for
- * implementations of {@link TableModificationRel}.
+ * Relational expression that modifies a table.
+ *
+ * It is similar to {@link org.apache.calcite.rel.core.TableScan},
+ * but represents a request to modify a table rather than read from it.
+ * It takes one child which produces the modified rows. Those rows are:
+ *
+ * <ul>
+ * <li>For {@code INSERT}, those rows are the new values;
+ * <li>for {@code DELETE}, the old values;
+ * <li>for {@code UPDATE}, all old values plus updated new values.
+ * </ul>
  */
-public abstract class TableModificationRelBase extends SingleRel {
+public abstract class TableModify extends SingleRel {
   //~ Enums ------------------------------------------------------------------
 
   /**
@@ -58,7 +75,7 @@ public abstract class TableModificationRelBase extends SingleRel {
 
   //~ Constructors -----------------------------------------------------------
 
-  protected TableModificationRelBase(
+  protected TableModify(
       RelOptCluster cluster,
       RelTraitSet traits,
       RelOptTable table,
@@ -181,4 +198,4 @@ public abstract class TableModificationRelBase extends SingleRel {
   }
 }
 
-// End TableModificationRelBase.java
+// End TableModify.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/core/TableScan.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/TableScan.java b/core/src/main/java/org/apache/calcite/rel/core/TableScan.java
index 7532080..9eaa931 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/TableScan.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/TableScan.java
@@ -14,22 +14,34 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel;
-
-import java.util.*;
-
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.rex.RexBuilder;
-import org.eigenbase.rex.RexNode;
-
-import net.hydromatic.optiq.util.BitSets;
+package org.apache.calcite.rel.core;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.AbstractRelNode;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelInput;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelShuttle;
+import org.apache.calcite.rel.RelWriter;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.util.BitSets;
+
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.List;
+import java.util.Set;
 
 /**
- * <code>TableAccessRelBase</code> is an abstract base class for implementations
- * of {@link TableAccessRel}.
+ * Relational operator that returns the contents of a table.
  */
-public abstract class TableAccessRelBase extends AbstractRelNode {
+public abstract class TableScan extends AbstractRelNode {
   //~ Instance fields --------------------------------------------------------
 
   /**
@@ -39,7 +51,7 @@ public abstract class TableAccessRelBase extends AbstractRelNode {
 
   //~ Constructors -----------------------------------------------------------
 
-  protected TableAccessRelBase(
+  protected TableScan(
       RelOptCluster cluster,
       RelTraitSet traits,
       RelOptTable table) {
@@ -51,9 +63,9 @@ public abstract class TableAccessRelBase extends AbstractRelNode {
   }
 
   /**
-   * Creates a TableAccessRelBase by parsing serialized output.
+   * Creates a TableScan by parsing serialized output.
    */
-  protected TableAccessRelBase(RelInput input) {
+  protected TableScan(RelInput input) {
     this(input.getCluster(), input.getTraitSet(), input.getTable("table"));
   }
 
@@ -96,7 +108,8 @@ public abstract class TableAccessRelBase extends AbstractRelNode {
    * fields that were not included in the table's official type.
    *
    * <p>The default implementation assumes that tables cannot do either of
-   * these operations, therefore it adds a {@link ProjectRel}, projecting
+   * these operations, therefore it adds a
+   * {@link org.apache.calcite.rel.logical.LogicalProject}, projecting
    * {@code NULL} values for the extra fields.</p>
    *
    * <p>Sub-classes, representing table types that have these capabilities,
@@ -140,10 +153,9 @@ public abstract class TableAccessRelBase extends AbstractRelNode {
     return projectFactory.createProject(this, exprList, nameList);
   }
 
-  @Override
-  public RelNode accept(RelShuttle shuttle) {
+  @Override public RelNode accept(RelShuttle shuttle) {
     return shuttle.visit(this);
   }
 }
 
-// End TableAccessRelBase.java
+// End TableScan.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/core/Uncollect.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/Uncollect.java b/core/src/main/java/org/apache/calcite/rel/core/Uncollect.java
index 3f8c61c..b31700b 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/Uncollect.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/Uncollect.java
@@ -14,26 +14,32 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel;
+package org.apache.calcite.rel.core;
 
-import java.util.List;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelInput;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.SingleRel;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.sql.SqlUtil;
 
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
+import java.util.List;
 
 /**
- * A relational expression which unnests its input's sole column into a
+ * Relational expression that unnests its input's sole column into a
  * relation.
  *
- * <p>Like its inverse operation {@link CollectRel}, UncollectRel is generally
- * invoked in a nested loop, driven by {@link CorrelatorRel} or similar.
+ * <p>Like its inverse operation {@link Collect}, Uncollect is generally
+ * invoked in a nested loop, driven by {@link Correlator} or similar.
  */
-public class UncollectRel extends SingleRel {
+public class Uncollect extends SingleRel {
   //~ Constructors -----------------------------------------------------------
 
   /**
-   * Creates an UncollectRel.
+   * Creates an Uncollect.
    *
    * <p>The row type of the child relational expression must contain precisely
    * one column, that column must be a multiset of records.
@@ -42,16 +48,16 @@ public class UncollectRel extends SingleRel {
    * @param traitSet Traits
    * @param child   Child relational expression
    */
-  public UncollectRel(RelOptCluster cluster, RelTraitSet traitSet,
+  public Uncollect(RelOptCluster cluster, RelTraitSet traitSet,
       RelNode child) {
     super(cluster, traitSet, child);
     assert deriveRowType() != null : "invalid child rowtype";
   }
 
   /**
-   * Creates an UncollectRel by parsing serialized output.
+   * Creates an Uncollect by parsing serialized output.
    */
-  public UncollectRel(RelInput input) {
+  public Uncollect(RelInput input) {
     this(input.getCluster(), input.getTraitSet(), input.getInput());
   }
 
@@ -64,11 +70,11 @@ public class UncollectRel extends SingleRel {
 
   public RelNode copy(RelTraitSet traitSet, RelNode input) {
     assert traitSet.containsIfApplicable(Convention.NONE);
-    return new UncollectRel(getCluster(), traitSet, input);
+    return new Uncollect(getCluster(), traitSet, input);
   }
 
   protected RelDataType deriveRowType() {
-    return deriveUncollectRowType(getChild());
+    return deriveUncollectRowType(getInput());
   }
 
   /**
@@ -96,4 +102,4 @@ public class UncollectRel extends SingleRel {
   }
 }
 
-// End UncollectRel.java
+// End Uncollect.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/core/Union.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/Union.java b/core/src/main/java/org/apache/calcite/rel/core/Union.java
index 4d31b71..1907199 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/Union.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/Union.java
@@ -14,22 +14,27 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel;
+package org.apache.calcite.rel.core;
 
-import java.util.List;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelInput;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.sql.SqlKind;
 
-import org.eigenbase.rel.metadata.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.sql.SqlKind;
+import java.util.List;
 
 /**
- * <code>UnionRelBase</code> is an abstract base class for implementations of
- * {@link UnionRel}.
+ * Relational expression that returns the union of the rows of its inputs,
+ * optionally eliminating duplicates.
+ *
+ * <p>Corresponds to SQL {@code UNION} and {@code UNION ALL}.
  */
-public abstract class UnionRelBase extends SetOpRel {
+public abstract class Union extends SetOp {
   //~ Constructors -----------------------------------------------------------
 
-  protected UnionRelBase(
+  protected Union(
       RelOptCluster cluster,
       RelTraitSet traits,
       List<RelNode> inputs,
@@ -38,9 +43,9 @@ public abstract class UnionRelBase extends SetOpRel {
   }
 
   /**
-   * Creates a UnionRelBase by parsing serialized output.
+   * Creates a Union by parsing serialized output.
    */
-  protected UnionRelBase(RelInput input) {
+  protected Union(RelInput input) {
     super(input);
   }
 
@@ -70,4 +75,4 @@ public abstract class UnionRelBase extends SetOpRel {
   }
 }
 
-// End UnionRelBase.java
+// End Union.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/core/Values.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/Values.java b/core/src/main/java/org/apache/calcite/rel/core/Values.java
index 4312bab..6f1458e 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/Values.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/Values.java
@@ -14,26 +14,32 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel;
-
-import java.util.*;
-
-import org.eigenbase.rel.metadata.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.rex.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.util.Pair;
-
-import net.hydromatic.linq4j.function.Function1;
-import net.hydromatic.linq4j.function.Functions;
+package org.apache.calcite.rel.core;
+
+import org.apache.calcite.linq4j.function.Function1;
+import org.apache.calcite.linq4j.function.Functions;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.AbstractRelNode;
+import org.apache.calcite.rel.RelInput;
+import org.apache.calcite.rel.RelWriter;
+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.RexLiteral;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.type.SqlTypeUtil;
+import org.apache.calcite.util.Pair;
+
+import java.util.List;
 
 /**
- * <code>ValuesRelBase</code> is an abstract base class for implementations of
- * {@link ValuesRel}.
+ * Relational expression whose value is a sequence of zero or more literal row
+ * values.
  */
-public abstract class ValuesRelBase extends AbstractRelNode {
+public abstract class Values extends AbstractRelNode {
   /**
    * Lambda that helps render tuples as strings.
    */
@@ -54,17 +60,19 @@ public abstract class ValuesRelBase extends AbstractRelNode {
   //~ Constructors -----------------------------------------------------------
 
   /**
-   * Creates a new ValuesRelBase. Note that tuples passed in become owned by
+   * Creates a new Values.
+   *
+   * <p>Note that tuples passed in become owned by
    * this rel (without a deep copy), so caller must not modify them after this
    * call, otherwise bad things will happen.
    *
-   * @param cluster .
-   * @param rowType row type for tuples produced by this rel
+   * @param cluster Cluster that this relational expression belongs to
+   * @param rowType Row type for tuples produced by this rel
    * @param tuples  2-dimensional array of tuple values to be produced; outer
    *                list contains tuples; each inner list is one tuple; all
    *                tuples must be of same length, conforming to rowType
    */
-  protected ValuesRelBase(
+  protected Values(
       RelOptCluster cluster,
       RelDataType rowType,
       List<List<RexLiteral>> tuples,
@@ -76,11 +84,10 @@ public abstract class ValuesRelBase extends AbstractRelNode {
   }
 
   /**
-   * Creates a ValuesRelBase by parsing serialized output.
+   * Creates a Values by parsing serialized output.
    */
-  public ValuesRelBase(RelInput input) {
-    this(
-        input.getCluster(), input.getRowType("type"),
+  public Values(RelInput input) {
+    this(input.getCluster(), input.getRowType("type"),
         input.getTuples("tuples"), input.getTraitSet());
   }
 
@@ -159,4 +166,4 @@ public abstract class ValuesRelBase extends AbstractRelNode {
   }
 }
 
-// End ValuesRelBase.java
+// End Values.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 e696749..043da29 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
@@ -14,37 +14,52 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel;
+package org.apache.calcite.rel.core;
 
-import java.util.*;
-
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.rex.*;
-import org.eigenbase.sql.SqlAggFunction;
-import org.eigenbase.util.ImmutableIntList;
-import org.eigenbase.util.Util;
-
-import net.hydromatic.linq4j.Ord;
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollationImpl;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelWriter;
+import org.apache.calcite.rel.SingleRel;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexChecker;
+import org.apache.calcite.rex.RexFieldCollation;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexLocalRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexSlot;
+import org.apache.calcite.rex.RexWindowBound;
+import org.apache.calcite.sql.SqlAggFunction;
+import org.apache.calcite.util.ImmutableIntList;
+import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
 
+import java.util.AbstractList;
+import java.util.BitSet;
+import java.util.List;
+
 /**
  * A relational expression representing a set of window aggregates.
  *
- * <p>A window rel can handle several window aggregate functions, over several
+ * <p>A Window can handle several window aggregate functions, over several
  * partitions, with pre- and post-expressions, and an optional post-filter.
  * Each of the partitions is defined by a partition key (zero or more columns)
  * and a range (logical or physical). The partitions expect the data to be
  * sorted correctly on input to the relational expression.
  *
- * <p>Each {@link org.eigenbase.rel.WindowRelBase.Window} has a set of
- * {@link org.eigenbase.rex.RexOver} objects.
+ * <p>Each {@link Window.Group} has a set of
+ * {@link org.apache.calcite.rex.RexOver} objects.
  *
- * <p>Created by {@link org.eigenbase.rel.rules.WindowedAggSplitterRule}.
+ * <p>Created by {@link org.apache.calcite.rel.rules.ProjectToWindowRule}.
  */
-public abstract class WindowRelBase extends SingleRel {
-  public final ImmutableList<Window> windows;
+public abstract class Window extends SingleRel {
+  public final ImmutableList<Group> groups;
   public final List<RexLiteral> constants;
 
   /**
@@ -54,38 +69,35 @@ public abstract class WindowRelBase extends SingleRel {
    * @param child   Input relational expression
    * @param constants List of constants that are additional inputs
    * @param rowType Output row type
-   * @param windows Windows
+   * @param groups Windows
    */
-  public WindowRelBase(
+  public Window(
       RelOptCluster cluster, RelTraitSet traits, RelNode child,
-      List<RexLiteral> constants, RelDataType rowType, List<Window> windows) {
+      List<RexLiteral> constants, RelDataType rowType, List<Group> groups) {
     super(cluster, traits, child);
     this.constants = ImmutableList.copyOf(constants);
     assert rowType != null;
     this.rowType = rowType;
-    this.windows = ImmutableList.copyOf(windows);
+    this.groups = ImmutableList.copyOf(groups);
   }
 
-  @Override
-  public boolean isValid(boolean fail) {
+  @Override public boolean isValid(boolean fail) {
     // In the window specifications, an aggregate call such as
     // 'SUM(RexInputRef #10)' refers to expression #10 of inputProgram.
     // (Not its projections.)
-    final RelDataType childRowType = getChild().getRowType();
+    final RelDataType childRowType = getInput().getRowType();
 
     final int childFieldCount = childRowType.getFieldCount();
     final int inputSize = childFieldCount + constants.size();
     final List<RelDataType> inputTypes =
         new AbstractList<RelDataType>() {
-          @Override
-          public RelDataType get(int index) {
+          @Override public RelDataType get(int index) {
             return index < childFieldCount
                 ? childRowType.getFieldList().get(index).getType()
                 : constants.get(index - childFieldCount).getType();
           }
 
-          @Override
-          public int size() {
+          @Override public int size() {
             return inputSize;
           }
         };
@@ -93,8 +105,8 @@ public abstract class WindowRelBase extends SingleRel {
     final RexChecker checker =
         new RexChecker(inputTypes, fail);
     int count = 0;
-    for (Window window : windows) {
-      for (RexWinAggCall over : window.aggCalls) {
+    for (Group group : groups) {
+      for (RexWinAggCall over : group.aggCalls) {
         ++count;
         if (!checker.isValid(over)) {
           return false;
@@ -110,13 +122,13 @@ public abstract class WindowRelBase extends SingleRel {
 
   public RelWriter explainTerms(RelWriter pw) {
     super.explainTerms(pw);
-    for (Ord<Window> window : Ord.zip(windows)) {
+    for (Ord<Group> window : Ord.zip(groups)) {
       pw.item("window#" + window.i, window.e.toString());
     }
     return pw;
   }
 
-  static ImmutableIntList getProjectOrdinals(final List<RexNode> exprs) {
+  public static ImmutableIntList getProjectOrdinals(final List<RexNode> exprs) {
     return ImmutableIntList.copyOf(
         new AbstractList<Integer>() {
           public Integer get(int index) {
@@ -129,7 +141,8 @@ public abstract class WindowRelBase extends SingleRel {
         });
   }
 
-  static RelCollation getCollation(final List<RexFieldCollation> collations) {
+  public static RelCollation getCollation(
+      final List<RexFieldCollation> collations) {
     return RelCollationImpl.of(
         new AbstractList<RelFieldCollation>() {
           public RelFieldCollation get(int index) {
@@ -155,8 +168,10 @@ public abstract class WindowRelBase extends SingleRel {
   }
 
   /**
-   * A Window is a range of input rows, defined by an upper and lower bound.
-   * It also has zero or more partitioning columns.
+   * Group of windowed aggregate calls that have the same window specification.
+   *
+   * <p>The specification is defined by an upper and lower bound, and
+   * also has zero or more partitioning columns.
    *
    * <p>A window is either logical or physical. A physical window is measured
    * in terms of row count. A logical window is measured in terms of rows
@@ -174,8 +189,8 @@ public abstract class WindowRelBase extends SingleRel {
    * CURRENT ROW</code>) is a logical window with an upper and lower bound.
    * </ul>
    */
-  public static class Window {
-    public final BitSet groupSet;
+  public static class Group {
+    public final BitSet keys;
     public final boolean isRows;
     public final RexWindowBound lowerBound;
     public final RexWindowBound upperBound;
@@ -183,22 +198,22 @@ public abstract class WindowRelBase extends SingleRel {
     private final String digest;
 
     /**
-     * List of {@link org.eigenbase.rel.WindowRelBase.RexWinAggCall}
+     * List of {@link Window.RexWinAggCall}
      * objects, each of which is a call to a
-     * {@link org.eigenbase.sql.SqlAggFunction}.
+     * {@link org.apache.calcite.sql.SqlAggFunction}.
      */
     public final ImmutableList<RexWinAggCall> aggCalls;
 
-    public Window(
-        BitSet groupSet,
+    public Group(
+        BitSet keys,
         boolean isRows,
         RexWindowBound lowerBound,
         RexWindowBound upperBound,
         RelCollation orderKeys,
         List<RexWinAggCall> aggCalls) {
       assert orderKeys != null : "precondition: ordinals != null";
-      assert groupSet != null;
-      this.groupSet = groupSet;
+      assert keys != null;
+      this.keys = keys;
       this.isRows = isRows;
       this.lowerBound = lowerBound;
       this.upperBound = upperBound;
@@ -214,7 +229,7 @@ public abstract class WindowRelBase extends SingleRel {
     private String computeString() {
       final StringBuilder buf = new StringBuilder();
       buf.append("window(partition ");
-      buf.append(groupSet);
+      buf.append(keys);
       buf.append(" order by ");
       buf.append(orderKeys);
       buf.append(isRows ? " rows " : " range ");
@@ -236,15 +251,13 @@ public abstract class WindowRelBase extends SingleRel {
       return buf.toString();
     }
 
-    @Override
-    public boolean equals(Object obj) {
+    @Override public boolean equals(Object obj) {
       return this == obj
-          || obj instanceof Window
-          && this.digest.equals(((Window) obj).digest);
+          || obj instanceof Group
+          && this.digest.equals(((Group) obj).digest);
     }
 
-    @Override
-    public int hashCode() {
+    @Override public int hashCode() {
       return digest.hashCode();
     }
 
@@ -257,9 +270,9 @@ public abstract class WindowRelBase extends SingleRel {
      * This is useful to refine data type of window aggregates.
      * For instance sum(non-nullable) over (empty window) is NULL.
      * @return true when the window is non-empty
-     * @see org.eigenbase.sql.SqlWindow#isAlwaysNonEmpty()
-     * @see org.eigenbase.sql.SqlOperatorBinding#getGroupCount()
-     * @see org.eigenbase.sql.validate.SqlValidatorImpl#resolveWindow(org.eigenbase.sql.SqlNode, org.eigenbase.sql.validate.SqlValidatorScope, boolean)
+     * @see org.apache.calcite.sql.SqlWindow#isAlwaysNonEmpty()
+     * @see org.apache.calcite.sql.SqlOperatorBinding#getGroupCount()
+     * @see org.apache.calcite.sql.validate.SqlValidatorImpl#resolveWindow(org.apache.calcite.sql.SqlNode, org.apache.calcite.sql.validate.SqlValidatorScope, boolean)
      */
     public boolean isAlwaysNonEmpty() {
       int lowerKey = lowerBound.getOrderKey();
@@ -271,10 +284,10 @@ public abstract class WindowRelBase extends SingleRel {
      * Presents a view of the {@link RexWinAggCall} list as a list of
      * {@link AggregateCall}.
      */
-    public List<AggregateCall> getAggregateCalls(WindowRelBase windowRel) {
+    public List<AggregateCall> getAggregateCalls(Window windowRel) {
       final List<String> fieldNames =
           Util.skip(windowRel.getRowType().getFieldNames(),
-              windowRel.getChild().getRowType().getFieldCount());
+              windowRel.getInput().getRowType().getFieldCount());
       return new AbstractList<AggregateCall>() {
         public int size() {
           return aggCalls.size();
@@ -283,7 +296,7 @@ public abstract class WindowRelBase extends SingleRel {
         public AggregateCall get(int index) {
           final RexWinAggCall aggCall = aggCalls.get(index);
           return new AggregateCall(
-              (Aggregation) aggCall.getOperator(),
+              (SqlAggFunction) aggCall.getOperator(),
               false,
               getProjectOrdinals(aggCall.getOperands()),
               aggCall.getType(),
@@ -296,10 +309,10 @@ public abstract class WindowRelBase extends SingleRel {
   /**
    * A call to a windowed aggregate function.
    *
-   * <p>Belongs to a {@link org.eigenbase.rel.WindowRelBase.Window}.
+   * <p>Belongs to a {@link Window.Group}.
    *
-   * <p>It's a bastard son of a {@link org.eigenbase.rex.RexCall}; similar
-   * enough that it gets visited by a {@link org.eigenbase.rex.RexVisitor},
+   * <p>It's a bastard son of a {@link org.apache.calcite.rex.RexCall}; similar
+   * enough that it gets visited by a {@link org.apache.calcite.rex.RexVisitor},
    * but it also has some extra data members.
    */
   public static class RexWinAggCall extends RexCall {
@@ -325,11 +338,10 @@ public abstract class WindowRelBase extends SingleRel {
       this.ordinal = ordinal;
     }
 
-    @Override
-    public RexCall clone(RelDataType type, List<RexNode> operands) {
+    @Override public RexCall clone(RelDataType type, List<RexNode> operands) {
       throw new UnsupportedOperationException();
     }
   }
 }
 
-// End WindowRelBase.java
+// End Window.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/core/package-info.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/package-info.java b/core/src/main/java/org/apache/calcite/rel/core/package-info.java
new file mode 100644
index 0000000..2e547fb
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/rel/core/package-info.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Defines core relational expressions.
+ *
+ * <h2>Related packages and classes</h2>
+ * <ul>
+ *
+ * <li>Package <code>
+ * <a href="../logical/package-summary.html">org.apache.calcite.rel.logical</a></code>
+ * contains logical relational expressions
+ *
+ * <li>Package <code>
+ * <a href="../package-summary.html">org.apache.calcite.rex</a></code>
+ * defines the relational expression API
+ *
+ * </ul>
+ */
+package org.apache.calcite.rel.core;
+
+// End package-info.java


[25/58] [abbrv] [partial] incubator-calcite git commit: [CALCITE-306] Standardize code style for "import package.*; "

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/fun/SqlMultisetQueryConstructor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlMultisetQueryConstructor.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlMultisetQueryConstructor.java
index f06d744..9e50507 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlMultisetQueryConstructor.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlMultisetQueryConstructor.java
@@ -14,16 +14,27 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.fun;
+package org.apache.calcite.sql.fun;
 
-import java.util.List;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlCallBinding;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlOperatorBinding;
+import org.apache.calcite.sql.SqlSelect;
+import org.apache.calcite.sql.SqlSpecialOperator;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.ReturnTypes;
+import org.apache.calcite.sql.type.SqlTypeUtil;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql.validate.SqlValidatorNamespace;
+import org.apache.calcite.sql.validate.SqlValidatorScope;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.sql.validate.*;
+import java.util.List;
 
-import static org.eigenbase.util.Static.RESOURCE;
+import static org.apache.calcite.util.Static.RESOURCE;
 
 /**
  * Definition of the SQL:2003 standard MULTISET query constructor, <code>

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/fun/SqlMultisetSetOperator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlMultisetSetOperator.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlMultisetSetOperator.java
index c8cbb1d..c997d3f 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlMultisetSetOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlMultisetSetOperator.java
@@ -14,17 +14,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.fun;
+package org.apache.calcite.sql.fun;
 
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.type.*;
+import org.apache.calcite.sql.SqlBinaryOperator;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.type.InferTypes;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.ReturnTypes;
 
 /**
  * An operator which performs set operations on multisets, such as "MULTISET
  * UNION ALL".
  *
- * <p>Not to be confused with {@link SqlMultisetValueConstructor} or {@link
- * SqlMultisetQueryConstructor}.
+ * <p>Not to be confused with {@link SqlMultisetValueConstructor} or
+ * {@link SqlMultisetQueryConstructor}.
  *
  * <p>todo: Represent the ALL keyword to MULTISET UNION ALL etc. as a hidden
  * operand. Then we can obsolete this class.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/fun/SqlMultisetValueConstructor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlMultisetValueConstructor.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlMultisetValueConstructor.java
index 4339690..c407f58 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlMultisetValueConstructor.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlMultisetValueConstructor.java
@@ -14,15 +14,24 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.fun;
+package org.apache.calcite.sql.fun;
 
-import java.util.List;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlCallBinding;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperatorBinding;
+import org.apache.calcite.sql.SqlSpecialOperator;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.ReturnTypes;
+import org.apache.calcite.sql.type.SqlTypeUtil;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.type.*;
+import java.util.List;
 
-import static org.eigenbase.util.Static.RESOURCE;
+import static org.apache.calcite.util.Static.RESOURCE;
 
 /**
  * Definition of the SQL:2003 standard MULTISET constructor, <code>MULTISET

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/fun/SqlNewOperator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlNewOperator.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlNewOperator.java
index 61618c8..ca24298 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlNewOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlNewOperator.java
@@ -14,10 +14,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.fun;
+package org.apache.calcite.sql.fun;
 
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.validate.*;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlPrefixOperator;
+import org.apache.calcite.sql.validate.SqlValidator;
 
 /**
  * SqlNewOperator represents an SQL <code>new specification</code> such as

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/fun/SqlNtileAggFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlNtileAggFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlNtileAggFunction.java
index 9364e10..5aaa2c3 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlNtileAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlNtileAggFunction.java
@@ -14,17 +14,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.fun;
+package org.apache.calcite.sql.fun;
 
-import java.util.List;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.sql.SqlAggFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.ReturnTypes;
 
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.reltype.RelDataTypeFactory;
-import org.eigenbase.sql.SqlAggFunction;
-import org.eigenbase.sql.SqlFunctionCategory;
-import org.eigenbase.sql.SqlKind;
-import org.eigenbase.sql.type.OperandTypes;
-import org.eigenbase.sql.type.ReturnTypes;
+import java.util.List;
 
 /**
  * <code>NTILE</code> aggregate function

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/fun/SqlNullifFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlNullifFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlNullifFunction.java
index 22bb3f1..c5529f0 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlNullifFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlNullifFunction.java
@@ -14,14 +14,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.fun;
+package org.apache.calcite.sql.fun;
 
-import java.util.List;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlNodeList;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.ReturnTypes;
+import org.apache.calcite.sql.validate.SqlValidator;
 
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.sql.validate.*;
+import java.util.List;
 
 /**
  * The <code>NULLIF</code> function.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/fun/SqlOverlapsOperator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlOverlapsOperator.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlOverlapsOperator.java
index 888a6b5..e3643c1 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlOverlapsOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlOverlapsOperator.java
@@ -14,12 +14,23 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.fun;
-
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.sql.validate.*;
+package org.apache.calcite.sql.fun;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlCallBinding;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlOperandCountRange;
+import org.apache.calcite.sql.SqlSpecialOperator;
+import org.apache.calcite.sql.SqlUtil;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.type.InferTypes;
+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.sql.validate.SqlValidator;
+import org.apache.calcite.sql.validate.SqlValidatorScope;
 
 import com.google.common.collect.ImmutableList;
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/fun/SqlOverlayFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlOverlayFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlOverlayFunction.java
index 462fbbd..1739332 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlOverlayFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlOverlayFunction.java
@@ -14,10 +14,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.fun;
+package org.apache.calcite.sql.fun;
 
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.type.*;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.ReturnTypes;
+import org.apache.calcite.sql.type.SqlOperandTypeChecker;
 
 /**
  * The <code>OVERLAY</code> function.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/fun/SqlPositionFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlPositionFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlPositionFunction.java
index 6046997..f85e746 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlPositionFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlPositionFunction.java
@@ -14,10 +14,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.fun;
+package org.apache.calcite.sql.fun;
 
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.type.*;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlCallBinding;
+import org.apache.calcite.sql.SqlFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.ReturnTypes;
 
 /**
  * The <code>POSITION</code> function.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/fun/SqlRowOperator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlRowOperator.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlRowOperator.java
index b97f466..63ab3e9 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlRowOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlRowOperator.java
@@ -14,16 +14,23 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.fun;
+package org.apache.calcite.sql.fun;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlOperatorBinding;
+import org.apache.calcite.sql.SqlSpecialOperator;
+import org.apache.calcite.sql.SqlSyntax;
+import org.apache.calcite.sql.SqlUtil;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.type.InferTypes;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.util.Pair;
 
 import java.util.AbstractList;
 import java.util.Map;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.util.Pair;
-
 /**
  * SqlRowOperator represents the special ROW constructor.
  *

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/fun/SqlSingleValueAggFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlSingleValueAggFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlSingleValueAggFunction.java
index b847b63..f13f3d8 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlSingleValueAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlSingleValueAggFunction.java
@@ -14,16 +14,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.fun;
+package org.apache.calcite.sql.fun;
 
-import java.util.List;
-
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.type.*;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.sql.SqlAggFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.ReturnTypes;
 
 import com.google.common.collect.ImmutableList;
 
+import java.util.List;
+
 /**
  * <code>SINGLE_VALUE</code> aggregate function returns the input value if there
  * is only one value in the input; Otherwise it triggers a run-time error.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java
index 5a75975..465fcc9 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java
@@ -14,18 +14,46 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.fun;
-
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.sql.util.*;
+package org.apache.calcite.sql.fun;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlAggFunction;
+import org.apache.calcite.sql.SqlAsOperator;
+import org.apache.calcite.sql.SqlBinaryOperator;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlFunctionalOperator;
+import org.apache.calcite.sql.SqlInternalOperator;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlOperandCountRange;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlOverOperator;
+import org.apache.calcite.sql.SqlPostfixOperator;
+import org.apache.calcite.sql.SqlPrefixOperator;
+import org.apache.calcite.sql.SqlProcedureCallOperator;
+import org.apache.calcite.sql.SqlRankFunction;
+import org.apache.calcite.sql.SqlSampleSpec;
+import org.apache.calcite.sql.SqlSetOperator;
+import org.apache.calcite.sql.SqlSpecialOperator;
+import org.apache.calcite.sql.SqlUnnestOperator;
+import org.apache.calcite.sql.SqlUtil;
+import org.apache.calcite.sql.SqlValuesOperator;
+import org.apache.calcite.sql.SqlWindow;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.type.InferTypes;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.ReturnTypes;
+import org.apache.calcite.sql.type.SqlOperandCountRanges;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.util.ReflectiveSqlOperatorTable;
 
 import com.google.common.collect.ImmutableList;
 
 /**
- * Implementation of {@link org.eigenbase.sql.SqlOperatorTable} containing the
- * standard operators and functions.
+ * Implementation of {@link org.apache.calcite.sql.SqlOperatorTable} containing
+ * the standard operators and functions.
  */
 public class SqlStdOperatorTable extends ReflectiveSqlOperatorTable {
   //~ Static fields/initializers ---------------------------------------------
@@ -964,9 +992,9 @@ public class SqlStdOperatorTable extends ReflectiveSqlOperatorTable {
    * <p>Operands are as follows:
    *
    * <ol>
-   * <li>name of window function ({@link org.eigenbase.sql.SqlCall})</li>
-   * <li>window name ({@link org.eigenbase.sql.SqlLiteral}) or window in-line
-   * specification (@link SqlWindowOperator})</li>
+   * <li>name of window function ({@link org.apache.calcite.sql.SqlCall})</li>
+   * <li>window name ({@link org.apache.calcite.sql.SqlLiteral}) or window
+   * in-line specification (@link SqlWindowOperator})</li>
    * </ol>
    */
   public static final SqlBinaryOperator OVER = new SqlOverOperator();

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/fun/SqlStringContextVariable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlStringContextVariable.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlStringContextVariable.java
index c1f641e..db7057e 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlStringContextVariable.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlStringContextVariable.java
@@ -14,11 +14,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.fun;
+package org.apache.calcite.sql.fun;
 
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.sql.validate.*;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlSyntax;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.ReturnTypes;
+import org.apache.calcite.sql.validate.SqlMonotonicity;
+import org.apache.calcite.sql.validate.SqlValidatorScope;
 
 /**
  * Base class for functions such as "USER", "CURRENT_ROLE", and "CURRENT_PATH".

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/fun/SqlSubstringFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlSubstringFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlSubstringFunction.java
index bf84442..5cac08f 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlSubstringFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlSubstringFunction.java
@@ -14,20 +14,34 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.fun;
-
-import java.math.*;
-import java.util.List;
-
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.sql.validate.*;
-
-import net.hydromatic.linq4j.Ord;
+package org.apache.calcite.sql.fun;
+
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlCallBinding;
+import org.apache.calcite.sql.SqlFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperandCountRange;
+import org.apache.calcite.sql.SqlUtil;
+import org.apache.calcite.sql.SqlWriter;
+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.SqlTypeName;
+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 com.google.common.collect.ImmutableList;
 
+import java.math.BigDecimal;
+import java.util.List;
+
 /**
  * Definition of the "SUBSTRING" builtin SQL function.
  */

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/fun/SqlSumAggFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlSumAggFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlSumAggFunction.java
index f70776b..9b2835d 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlSumAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlSumAggFunction.java
@@ -14,16 +14,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.fun;
+package org.apache.calcite.sql.fun;
 
-import java.util.List;
-
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.type.*;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.sql.SqlAggFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.ReturnTypes;
 
 import com.google.common.collect.ImmutableList;
 
+import java.util.List;
+
 /**
  * <code>Sum</code> is an aggregator which returns the sum of the values which
  * go into it. It has precisely one argument of numeric type (<code>int</code>,

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/fun/SqlSumEmptyIsZeroAggFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlSumEmptyIsZeroAggFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlSumEmptyIsZeroAggFunction.java
index 1cff679..b1d50b8 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlSumEmptyIsZeroAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlSumEmptyIsZeroAggFunction.java
@@ -14,16 +14,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.fun;
+package org.apache.calcite.sql.fun;
 
-import java.util.List;
-
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.type.*;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.sql.SqlAggFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.ReturnTypes;
+import org.apache.calcite.sql.type.SqlTypeName;
 
 import com.google.common.collect.ImmutableList;
 
+import java.util.List;
+
 /**
  * <code>Sum0</code> is an aggregator which returns the sum of the values which
  * go into it like <code>Sum</code>. It differs in that when no non null values

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/fun/SqlThrowOperator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlThrowOperator.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlThrowOperator.java
index 51c9793..f223141 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlThrowOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlThrowOperator.java
@@ -14,10 +14,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.fun;
+package org.apache.calcite.sql.fun;
 
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.type.*;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlInternalOperator;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.ReturnTypes;
 
 /**
  * An internal operator that throws an exception.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/fun/SqlTrimFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlTrimFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlTrimFunction.java
index 8ac8d93..f798e3e 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlTrimFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlTrimFunction.java
@@ -14,17 +14,29 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.fun;
+package org.apache.calcite.sql.fun;
+
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlCallBinding;
+import org.apache.calcite.sql.SqlFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.ReturnTypes;
+import org.apache.calcite.sql.type.SameOperandTypeChecker;
+import org.apache.calcite.sql.type.SqlTypeFamily;
+import org.apache.calcite.sql.type.SqlTypeTransforms;
+import org.apache.calcite.sql.type.SqlTypeUtil;
+
+import com.google.common.collect.ImmutableList;
 
 import java.util.Arrays;
 import java.util.List;
 
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.sql.type.*;
-
-import com.google.common.collect.ImmutableList;
-
 /**
  * Definition of the "TRIM" builtin SQL function.
  */
@@ -78,8 +90,8 @@ public class SqlTrimFunction extends SqlFunction {
                 SqlTypeFamily.ANY, SqlTypeFamily.STRING, SqlTypeFamily.STRING),
             // Arguments 1 and 2 must have same type
             new SameOperandTypeChecker(3) {
-              @Override
-              protected List<Integer> getOperandList(int operandCount) {
+              @Override protected List<Integer>
+              getOperandList(int operandCount) {
                 return ImmutableList.of(1, 2);
               }
             }),

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/fun/package-info.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/package-info.java b/core/src/main/java/org/apache/calcite/sql/fun/package-info.java
index 07b5db7..bd996ce 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/package-info.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/package-info.java
@@ -20,13 +20,13 @@
  * operators.
  *
  * <p>The standard set of row-level functions and operators are declared in
- * class {@link org.eigenbase.sql.fun.SqlStdOperatorTable}. Anonymous inner
+ * class {@link org.apache.calcite.sql.fun.SqlStdOperatorTable}. Anonymous inner
  * classes within that table are allowed only for specifying an operator's test
  * function; if other custom code is needed for an operator, it should be
  * implemented in a top-level class within this package instead.  Operators
  * which are not row-level (e.g. select and join) should be defined in package
- * {@link org.eigenbase.sql} instead.</p>
+ * {@link org.apache.calcite.sql} instead.</p>
  */
-package org.eigenbase.sql.fun;
+package org.apache.calcite.sql.fun;
 
 // End package-info.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/package-info.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/package-info.java b/core/src/main/java/org/apache/calcite/sql/package-info.java
index 5b45af4..698fd89 100644
--- a/core/src/main/java/org/apache/calcite/sql/package-info.java
+++ b/core/src/main/java/org/apache/calcite/sql/package-info.java
@@ -16,86 +16,82 @@
  */
 
 /**
- * Provides an SQL parser and object model. This package, and the dependent
- * <code>org.eigenbase.sql.parser</code> package, are independent of the other
- * saffron packages, so may be used standalone.<p>&nbsp;</p>
+ * Provides a SQL parser and object model.
+ *
+ * <p>This package, and the dependent <code>org.apache.calcite.sql.parser</code>
+ * package, are independent of the other Calcite packages, so may be used
+ * standalone.
  *
  * <h2>Parser</h2>
  *
- * <p>{@link org.eigenbase.sql.parser.SqlParser} parses a SQL string to a parse
- *     tree. It
- *     only performs the most basic syntactic validation.</p>
+ * <p>{@link org.apache.calcite.sql.parser.SqlParser} parses a SQL string to a
+ *     parse tree. It only performs the most basic syntactic validation.</p>
  *
  * <h2>Object model</h2>
  *
- * <p>Every node in the parse tree is a {@link org.eigenbase.sql.SqlNode}.
+ * <p>Every node in the parse tree is a {@link org.apache.calcite.sql.SqlNode}.
  *     Sub-types are:</p>
  * <ul>
  *
- *     <li>{@link org.eigenbase.sql.SqlLiteral} represents a boolean, numeric,
- *         string, or
- *         date constant, or the value <code>NULL</code>.
- *     </li>
+ *     <li>{@link org.apache.calcite.sql.SqlLiteral} represents a boolean,
+ *         numeric, string, or date constant, or the value <code>NULL</code>.
+ *         </li>
  *
- *     <li>{@link org.eigenbase.sql.SqlIdentifier} represents an identifier, such
- *         as <code>
- *             EMPNO</code> or <code>emp.deptno</code>.
- *     </li>
+ *     <li>{@link org.apache.calcite.sql.SqlIdentifier} represents an
+ *         identifier, such as <code> EMPNO</code> or <code>emp.deptno</code>.
+ *         </li>
  *
- *     <li>{@link org.eigenbase.sql.SqlCall} is a call to an operator or
+ *     <li>{@link org.apache.calcite.sql.SqlCall} is a call to an operator or
  *         function.  By means of special operators, we can use this construct
  *         to represent virtually every non-leaf node in the tree. For example,
  *         a <code>select</code> statement is a call to the 'select'
  *         operator.</li>
  *
- *     <li>{@link org.eigenbase.sql.SqlNodeList} is a list of nodes.</li>
+ *     <li>{@link org.apache.calcite.sql.SqlNodeList} is a list of nodes.</li>
  *
  * </ul>
  *
- * <p>A {@link org.eigenbase.sql.SqlOperator} describes the behavior of a node
- *     in the tree, such as how to un-parse a {@link org.eigenbase.sql.SqlCall}
- *     into a SQL string.  It is important to note that operators are metadata,
- *     not data: there is only one <code>SqlOperator</code> instance
- *     representing the '=' operator, even though there may be many calls to
- *     it.</p>
+ * <p>A {@link org.apache.calcite.sql.SqlOperator} describes the behavior of a
+ *     node in the tree, such as how to un-parse a
+ *     {@link org.apache.calcite.sql.SqlCall} into a SQL string.  It is
+ *     important to note that operators are metadata, not data: there is only
+ *     one <code>SqlOperator</code> instance representing the '=' operator, even
+ *     though there may be many calls to it.</p>
  *
  * <p><code>SqlOperator</code> has several derived classes which make it easy to
- *     define new operators: {@link org.eigenbase.sql.SqlFunction},
- *     {@link org.eigenbase.sql.SqlBinaryOperator},
- *     {@link org.eigenbase.sql.SqlPrefixOperator},
- *     {@link org.eigenbase.sql.SqlPostfixOperator}.
+ *     define new operators: {@link org.apache.calcite.sql.SqlFunction},
+ *     {@link org.apache.calcite.sql.SqlBinaryOperator},
+ *     {@link org.apache.calcite.sql.SqlPrefixOperator},
+ *     {@link org.apache.calcite.sql.SqlPostfixOperator}.
  *
  *     And there are singleton classes for special syntactic constructs
- *     {@link org.eigenbase.sql.SqlSelectOperator}
-
- *     and {@link org.eigenbase.sql.SqlJoin.SqlJoinOperator}. (These special operators
- *     even have their own sub-types of {@link org.eigenbase.sql.SqlCall}:
- *     {@link org.eigenbase.sql.SqlSelect}
- *     and {@link org.eigenbase.sql.SqlJoin}.)</p>
+ *     {@link org.apache.calcite.sql.SqlSelectOperator}
+ *     and {@link org.apache.calcite.sql.SqlJoin.SqlJoinOperator}. (These
+ *     special operators even have their own sub-types of
+ *     {@link org.apache.calcite.sql.SqlCall}:
+ *     {@link org.apache.calcite.sql.SqlSelect} and
+ *     {@link org.apache.calcite.sql.SqlJoin}.)</p>
  *
- * <p>A {@link org.eigenbase.sql.SqlOperatorTable} is a collection of operators. By
- *     supplying your own operator table, you can customize the dialect of SQL
- *     without
- *     modifying the parser.</p>
+ * <p>A {@link org.apache.calcite.sql.SqlOperatorTable} is a collection of
+ *     operators. By supplying your own operator table, you can customize the
+ *     dialect of SQL without modifying the parser.</p>
  *
  * <h2>Validation</h2>
  *
- * <p>{@link org.eigenbase.sql.validate.SqlValidator} checks that
- *     a tree of {@link org.eigenbase.sql.SqlNode}s is
- *     semantically valid. You supply a {@link org.eigenbase.sql.SqlOperatorTable}
- *     to
- *     describe the available functions and operators, and a
- *     {@link org.eigenbase.sql.validate.SqlValidatorCatalogReader}
- *     for access to the database's catalog.</p>
+ * <p>{@link org.apache.calcite.sql.validate.SqlValidator} checks that
+ *     a tree of {@link org.apache.calcite.sql.SqlNode}s is
+ *     semantically valid. You supply a
+ *     {@link org.apache.calcite.sql.SqlOperatorTable} to describe the available
+ *     functions and operators, and a
+ *     {@link org.apache.calcite.sql.validate.SqlValidatorCatalogReader} for
+ *     access to the database's catalog.</p>
  *
  * <h2>Generating SQL</h2>
  *
- * <p>A {@link org.eigenbase.sql.SqlWriter} converts a tree of {@link
- *     org.eigenbase.sql.SqlNode}s into a
- *     SQL string. A {@link org.eigenbase.sql.SqlDialect} defines how this
- *     happens.</p>
- *
+ * <p>A {@link org.apache.calcite.sql.SqlWriter} converts a tree of
+ * {@link org.apache.calcite.sql.SqlNode}s into a SQL string. A
+ * {@link org.apache.calcite.sql.SqlDialect} defines how this happens.</p>
  */
-package org.eigenbase.sql;
+package org.apache.calcite.sql;
 
 // End package-info.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 b9998c2..13f761b 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
@@ -14,22 +14,34 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.parser;
-
-import java.io.*;
-import java.lang.reflect.*;
-import java.util.*;
-
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.fun.*;
-import org.eigenbase.util.*;
-
-import net.hydromatic.avatica.Casing;
+package org.apache.calcite.sql.parser;
+
+import org.apache.calcite.avatica.Casing;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlSyntax;
+import org.apache.calcite.sql.SqlUnresolvedFunction;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Lists;
 
+import java.io.Reader;
+import java.io.StringReader;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+
 /**
  * Abstract base for parsers generated from CommonParser.jj.
  */

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/parser/SqlParseException.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/parser/SqlParseException.java b/core/src/main/java/org/apache/calcite/sql/parser/SqlParseException.java
index 3ae8072..62af5d9 100644
--- a/core/src/main/java/org/apache/calcite/sql/parser/SqlParseException.java
+++ b/core/src/main/java/org/apache/calcite/sql/parser/SqlParseException.java
@@ -14,18 +14,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.parser;
+package org.apache.calcite.sql.parser;
 
-import java.util.*;
+import org.apache.calcite.util.CalciteParserException;
 
-import org.eigenbase.util14.*;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Set;
+import java.util.TreeSet;
 
 /**
- * SqlParseException defines a checked exception corresponding to {@link
- * SqlParser}.
+ * SqlParseException defines a checked exception corresponding to
+ * {@link SqlParser}.
  */
 public class SqlParseException extends Exception
-    implements EigenbaseParserException {
+    implements CalciteParserException {
   //~ Instance fields --------------------------------------------------------
 
   private final SqlParserPos pos;
@@ -160,8 +163,8 @@ public class SqlParseException extends Exception
    * written during serialization.
    *
    * <p>SqlParseException is serializable but is not available on the client.
-   * This implementation converts this SqlParseException into a vanilla {@link
-   * RuntimeException} with the same message.
+   * This implementation converts this SqlParseException into a vanilla
+   * {@link RuntimeException} with the same message.
    */
   private Object writeReplace() {
     return new RuntimeException(

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/parser/SqlParser.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/parser/SqlParser.java b/core/src/main/java/org/apache/calcite/sql/parser/SqlParser.java
index 29c49dc..2555b5d 100644
--- a/core/src/main/java/org/apache/calcite/sql/parser/SqlParser.java
+++ b/core/src/main/java/org/apache/calcite/sql/parser/SqlParser.java
@@ -14,16 +14,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.parser;
+package org.apache.calcite.sql.parser;
 
-import java.io.*;
+import org.apache.calcite.avatica.Casing;
+import org.apache.calcite.avatica.Quoting;
+import org.apache.calcite.runtime.CalciteContextException;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.parser.impl.SqlParserImpl;
 
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.parser.impl.*;
-import org.eigenbase.util.*;
-
-import net.hydromatic.avatica.Casing;
-import net.hydromatic.avatica.Quoting;
+import java.io.StringReader;
 
 /**
  * A <code>SqlParser</code> parses a SQL statement.
@@ -101,9 +100,9 @@ public class SqlParser {
     try {
       return parser.parseSqlExpressionEof();
     } catch (Throwable ex) {
-      if ((ex instanceof EigenbaseContextException)
+      if ((ex instanceof CalciteContextException)
           && (originalInput != null)) {
-        ((EigenbaseContextException) ex).setOriginalStatement(
+        ((CalciteContextException) ex).setOriginalStatement(
             originalInput);
       }
       throw parser.normalizeException(ex);
@@ -113,8 +112,8 @@ public class SqlParser {
   /**
    * Parses a <code>SELECT</code> statement.
    *
-   * @return A {@link org.eigenbase.sql.SqlSelect} for a regular <code>
-   * SELECT</code> statement; a {@link org.eigenbase.sql.SqlBinaryOperator}
+   * @return A {@link org.apache.calcite.sql.SqlSelect} for a regular <code>
+   * SELECT</code> statement; a {@link org.apache.calcite.sql.SqlBinaryOperator}
    * for a <code>UNION</code>, <code>INTERSECT</code>, or <code>EXCEPT</code>.
    * @throws SqlParseException if there is a parse error
    */
@@ -122,9 +121,9 @@ public class SqlParser {
     try {
       return parser.parseSqlStmtEof();
     } catch (Throwable ex) {
-      if ((ex instanceof EigenbaseContextException)
+      if ((ex instanceof CalciteContextException)
           && (originalInput != null)) {
-        ((EigenbaseContextException) ex).setOriginalStatement(
+        ((CalciteContextException) ex).setOriginalStatement(
             originalInput);
       }
       throw parser.normalizeException(ex);
@@ -141,9 +140,9 @@ public class SqlParser {
     try {
       return parser.parseSqlStmtEof();
     } catch (Throwable ex) {
-      if ((ex instanceof EigenbaseContextException)
+      if ((ex instanceof CalciteContextException)
           && (originalInput != null)) {
-        ((EigenbaseContextException) ex).setOriginalStatement(
+        ((CalciteContextException) ex).setOriginalStatement(
             originalInput);
       }
       throw parser.normalizeException(ex);

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/parser/SqlParserImplFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/parser/SqlParserImplFactory.java b/core/src/main/java/org/apache/calcite/sql/parser/SqlParserImplFactory.java
index 2a271a1..6830af3 100644
--- a/core/src/main/java/org/apache/calcite/sql/parser/SqlParserImplFactory.java
+++ b/core/src/main/java/org/apache/calcite/sql/parser/SqlParserImplFactory.java
@@ -14,16 +14,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.parser;
+package org.apache.calcite.sql.parser;
 
 import java.io.Reader;
 
 /**
- * Factory for {@link org.eigenbase.sql.parser.SqlAbstractParserImpl} objects.
+ * Factory for
+ * {@link org.apache.calcite.sql.parser.SqlAbstractParserImpl} objects.
  *
  * <p>A parser factory allows you to include a custom parser in
- * {@link net.hydromatic.optiq.tools.Planner} created through
- * {@link net.hydromatic.optiq.tools.Frameworks}.</p>
+ * {@link org.apache.calcite.tools.Planner} created through
+ * {@link org.apache.calcite.tools.Frameworks}.</p>
  */
 public interface SqlParserImplFactory {
 
@@ -34,3 +35,5 @@ public interface SqlParserImplFactory {
    */
   SqlAbstractParserImpl getParser(Reader stream);
 }
+
+// End SqlParserImplFactory.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/parser/SqlParserPos.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/parser/SqlParserPos.java b/core/src/main/java/org/apache/calcite/sql/parser/SqlParserPos.java
index f4ddd3f..95d704c 100644
--- a/core/src/main/java/org/apache/calcite/sql/parser/SqlParserPos.java
+++ b/core/src/main/java/org/apache/calcite/sql/parser/SqlParserPos.java
@@ -14,14 +14,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.parser;
+package org.apache.calcite.sql.parser;
 
-import java.io.*;
-import java.util.*;
+import org.apache.calcite.sql.SqlNode;
 
-import org.eigenbase.sql.*;
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.List;
 
-import static org.eigenbase.util.Static.RESOURCE;
+import static org.apache.calcite.util.Static.RESOURCE;
 
 /**
  * SqlParserPos represents the position of a parsed token within SQL statement

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 1297f35..b184a86 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
@@ -14,22 +14,36 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.parser;
-
-import java.math.*;
-import java.nio.charset.*;
-import java.text.*;
-import java.util.*;
-import java.util.logging.*;
-
-import org.eigenbase.reltype.RelDataTypeSystem;
-import org.eigenbase.sql.*;
-import org.eigenbase.trace.*;
-import org.eigenbase.util.*;
-
-import net.hydromatic.avatica.Casing;
-
-import static org.eigenbase.util.Static.RESOURCE;
+package org.apache.calcite.sql.parser;
+
+import org.apache.calcite.avatica.Casing;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.runtime.CalciteContextException;
+import org.apache.calcite.sql.SqlBinaryOperator;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlIntervalLiteral;
+import org.apache.calcite.sql.SqlIntervalQualifier;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlNodeList;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlPostfixOperator;
+import org.apache.calcite.sql.SqlSpecialOperator;
+import org.apache.calcite.util.SaffronProperties;
+import org.apache.calcite.util.Util;
+import org.apache.calcite.util.trace.CalciteTrace;
+
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.charset.Charset;
+import java.text.SimpleDateFormat;
+import java.util.List;
+import java.util.Locale;
+import java.util.StringTokenizer;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import static org.apache.calcite.util.Static.RESOURCE;
 
 /**
  * Utility methods relating to parsing SQL.
@@ -37,7 +51,7 @@ import static org.eigenbase.util.Static.RESOURCE;
 public final class SqlParserUtil {
   //~ Static fields/initializers ---------------------------------------------
 
-  static final Logger LOGGER = EigenbaseTrace.getParserTracer();
+  static final Logger LOGGER = CalciteTrace.getParserTracer();
 
   //~ Constructors -----------------------------------------------------------
 
@@ -138,7 +152,7 @@ public final class SqlParserUtil {
       ret = intervalQualifier.evaluateIntervalLiteral(literal,
           intervalQualifier.getParserPosition(), RelDataTypeSystem.DEFAULT);
       assert ret != null;
-    } catch (EigenbaseContextException e) {
+    } catch (CalciteContextException e) {
       throw Util.newInternal(
           e, "while parsing day-to-second interval " + literal);
     }
@@ -180,7 +194,7 @@ public final class SqlParserUtil {
       ret = intervalQualifier.evaluateIntervalLiteral(literal,
           intervalQualifier.getParserPosition(), RelDataTypeSystem.DEFAULT);
       assert ret != null;
-    } catch (EigenbaseContextException e) {
+    } catch (CalciteContextException e) {
       throw Util.newInternal(
           e, "error parsing year-to-month interval " + literal);
     }
@@ -762,6 +776,7 @@ public final class SqlParserUtil {
 
   //~ Inner Classes ----------------------------------------------------------
 
+  /** The components of a collation definition, per the SQL standard. */
   public static class ParsedCollation {
     private final Charset charset;
     private final Locale locale;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/parser/impl/package-info.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/parser/impl/package-info.java b/core/src/main/java/org/apache/calcite/sql/parser/impl/package-info.java
index d5544b1..281cf49 100644
--- a/core/src/main/java/org/apache/calcite/sql/parser/impl/package-info.java
+++ b/core/src/main/java/org/apache/calcite/sql/parser/impl/package-info.java
@@ -17,8 +17,8 @@
 
 /**
  * Contains generated code for the
- * {@link org.eigenbase.sql.parser Eigenbase SQL parser}.
+ * {@link org.apache.calcite.sql.parser Calcite SQL parser}.
  */
-package org.eigenbase.sql.parser.impl;
+package org.apache.calcite.sql.parser.impl;
 
 // End package-info.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/parser/package-info.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/parser/package-info.java b/core/src/main/java/org/apache/calcite/sql/parser/package-info.java
index 0ee4abb..f518710 100644
--- a/core/src/main/java/org/apache/calcite/sql/parser/package-info.java
+++ b/core/src/main/java/org/apache/calcite/sql/parser/package-info.java
@@ -18,6 +18,6 @@
 /**
  * Provides a SQL parser.
  */
-package org.eigenbase.sql.parser;
+package org.apache.calcite.sql.parser;
 
 // End package-info.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/pretty/SqlFormatOptions.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/pretty/SqlFormatOptions.java b/core/src/main/java/org/apache/calcite/sql/pretty/SqlFormatOptions.java
index 01bc0a0..93a6eed 100644
--- a/core/src/main/java/org/apache/calcite/sql/pretty/SqlFormatOptions.java
+++ b/core/src/main/java/org/apache/calcite/sql/pretty/SqlFormatOptions.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.pretty;
+package org.apache.calcite.sql.pretty;
 
 /**
  * Data structure to hold options for

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 4eddfe8..d064e33 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
@@ -14,19 +14,28 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.pretty;
-
-import java.io.*;
-import java.lang.reflect.*;
-import java.util.*;
-import java.util.logging.*;
-
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.util.*;
-import org.eigenbase.trace.*;
-import org.eigenbase.util.*;
-
-import net.hydromatic.optiq.runtime.Spaces;
+package org.apache.calcite.sql.pretty;
+
+import org.apache.calcite.runtime.Spaces;
+import org.apache.calcite.sql.SqlDialect;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.util.SqlBuilder;
+import org.apache.calcite.sql.util.SqlString;
+import org.apache.calcite.util.Util;
+import org.apache.calcite.util.trace.CalciteLogger;
+
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.Stack;
+import java.util.logging.Logger;
 
 /**
  * Pretty printer for SQL statements.
@@ -90,10 +99,11 @@ import net.hydromatic.optiq.runtime.Spaces;
  * </tr>
  * <tr>
  * <td>{@link #setSubqueryStyle SubqueryStyle}</td>
- * <td>Style for formatting sub-queries. Values are: {@link
- * org.eigenbase.sql.SqlWriter.SubqueryStyle#HYDE Hyde}, {@link
- * org.eigenbase.sql.SqlWriter.SubqueryStyle#BLACK Black}.</td>
- * <td>{@link org.eigenbase.sql.SqlWriter.SubqueryStyle#HYDE Hyde}</td>
+ * <td>Style for formatting sub-queries. Values are:
+ * {@link org.apache.calcite.sql.SqlWriter.SubqueryStyle#HYDE Hyde},
+ * {@link org.apache.calcite.sql.SqlWriter.SubqueryStyle#BLACK Black}.</td>
+ *
+ * <td>{@link org.apache.calcite.sql.SqlWriter.SubqueryStyle#HYDE Hyde}</td>
  * </tr>
  * <tr>
  * <td>{@link #setLineLength LineLength}</td>
@@ -106,9 +116,9 @@ import net.hydromatic.optiq.runtime.Spaces;
 public class SqlPrettyWriter implements SqlWriter {
   //~ Static fields/initializers ---------------------------------------------
 
-  protected static final EigenbaseLogger LOGGER =
-      new EigenbaseLogger(
-          Logger.getLogger("org.eigenbase.sql.pretty.SqlPrettyWriter"));
+  protected static final CalciteLogger LOGGER =
+      new CalciteLogger(
+          Logger.getLogger("org.apache.calcite.sql.pretty.SqlPrettyWriter"));
 
   /**
    * Bean holding the default property values.
@@ -182,8 +192,8 @@ public class SqlPrettyWriter implements SqlWriter {
   }
 
   /**
-   * Sets the subquery style. Default is {@link
-   * org.eigenbase.sql.SqlWriter.SubqueryStyle#HYDE}.
+   * Sets the subquery style. Default is
+   * {@link org.apache.calcite.sql.SqlWriter.SubqueryStyle#HYDE}.
    */
   public void setSubqueryStyle(SubqueryStyle subqueryStyle) {
     this.subqueryStyle = subqueryStyle;
@@ -299,7 +309,7 @@ public class SqlPrettyWriter implements SqlWriter {
       String key = propertyNames[i];
       final Object value = bean.get(key);
       final Object defaultValue = DEFAULT_BEAN.get(key);
-      if (Util.equal(value, defaultValue)) {
+      if (com.google.common.base.Objects.equal(value, defaultValue)) {
         continue;
       }
       if (count++ > 0) {
@@ -751,7 +761,7 @@ public class SqlPrettyWriter implements SqlWriter {
     Util.pre(
         frame == this.frame,
         "Frame " + endedFrame.frameType
-        + " does not match current frame " + this.frame.frameType);
+            + " does not match current frame " + this.frame.frameType);
     if (this.frame == null) {
       throw new RuntimeException("No list started");
     }
@@ -959,7 +969,7 @@ public class SqlPrettyWriter implements SqlWriter {
   //~ Inner Classes ----------------------------------------------------------
 
   /**
-   * Implementation of {@link org.eigenbase.sql.SqlWriter.Frame}.
+   * Implementation of {@link org.apache.calcite.sql.SqlWriter.Frame}.
    */
   protected class FrameImpl implements Frame {
     final FrameType frameType;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/pretty/package-info.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/pretty/package-info.java b/core/src/main/java/org/apache/calcite/sql/pretty/package-info.java
index 71783c5..bd30861 100644
--- a/core/src/main/java/org/apache/calcite/sql/pretty/package-info.java
+++ b/core/src/main/java/org/apache/calcite/sql/pretty/package-info.java
@@ -18,6 +18,6 @@
 /**
  * Provides a pretty-printer for SQL statements.
  */
-package org.eigenbase.sql.pretty;
+package org.apache.calcite.sql.pretty;
 
 // End package-info.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/type/AbstractSqlType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/AbstractSqlType.java b/core/src/main/java/org/apache/calcite/sql/type/AbstractSqlType.java
index f9a798b..4baa04d 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/AbstractSqlType.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/AbstractSqlType.java
@@ -14,12 +14,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.type;
+package org.apache.calcite.sql.type;
 
-import java.io.*;
-import java.util.List;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFamily;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rel.type.RelDataTypeImpl;
+import org.apache.calcite.rel.type.RelDataTypePrecedenceList;
 
-import org.eigenbase.reltype.*;
+import java.io.Serializable;
+import java.util.List;
 
 /**
  * Abstract base class for SQL implementations of {@link RelDataType}.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/type/ArraySqlType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/ArraySqlType.java b/core/src/main/java/org/apache/calcite/sql/type/ArraySqlType.java
index 7794d52..8ab656e 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/ArraySqlType.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/ArraySqlType.java
@@ -14,10 +14,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.type;
+package org.apache.calcite.sql.type;
 
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.reltype.RelDataTypeFamily;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFamily;
 
 /**
  * SQL array type.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/type/AssignableOperandTypeChecker.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/AssignableOperandTypeChecker.java b/core/src/main/java/org/apache/calcite/sql/type/AssignableOperandTypeChecker.java
index 29574a6..3f9f9ba 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/AssignableOperandTypeChecker.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/AssignableOperandTypeChecker.java
@@ -14,18 +14,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.type;
+package org.apache.calcite.sql.type;
 
-import java.util.List;
-
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.util.Pair;
-
-import net.hydromatic.linq4j.Ord;
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlCallBinding;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperandCountRange;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.util.Pair;
 
 import com.google.common.collect.ImmutableList;
 
+import java.util.List;
+
 /**
  * AssignableOperandTypeChecker implements {@link SqlOperandTypeChecker} by
  * verifying that the type of each argument is assignable to a predefined set of

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 796f45b..949019f 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
@@ -14,13 +14,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.type;
+package org.apache.calcite.sql.type;
 
-import java.nio.charset.*;
+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 org.eigenbase.reltype.RelDataTypeSystem;
-import org.eigenbase.sql.*;
-import org.eigenbase.util.*;
+import java.nio.charset.Charset;
 
 /**
  * BasicSqlType represents a standard atomic SQL type (excluding interval

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/type/ComparableOperandTypeChecker.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/ComparableOperandTypeChecker.java b/core/src/main/java/org/apache/calcite/sql/type/ComparableOperandTypeChecker.java
index 0bf0029..454897d 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/ComparableOperandTypeChecker.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/ComparableOperandTypeChecker.java
@@ -14,12 +14,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.type;
+package org.apache.calcite.sql.type;
 
-import java.util.*;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeComparability;
+import org.apache.calcite.sql.SqlCallBinding;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlOperatorBinding;
+import org.apache.calcite.sql.SqlUtil;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
+import java.util.Collections;
 
 /**
  * Type checking strategy which verifies that types have the required attributes
@@ -77,8 +81,9 @@ public class ComparableOperandTypeChecker extends SameOperandTypeChecker {
   }
 
   /**
-   * Similar functionality to {@link #checkOperandTypes(SqlCallBinding,
-   * boolean)}, but not part of the interface, and cannot throw an error.
+   * Similar functionality to
+   * {@link #checkOperandTypes(SqlCallBinding, boolean)}, but not part of the
+   * interface, and cannot throw an error.
    */
   public boolean checkOperandTypes(
       SqlOperatorBinding callBinding) {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/type/CompositeOperandTypeChecker.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/CompositeOperandTypeChecker.java b/core/src/main/java/org/apache/calcite/sql/type/CompositeOperandTypeChecker.java
index 2046bc5..4c174ab 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/CompositeOperandTypeChecker.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/CompositeOperandTypeChecker.java
@@ -14,17 +14,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.type;
+package org.apache.calcite.sql.type;
 
-import java.util.*;
-
-import org.eigenbase.sql.*;
-import org.eigenbase.util.*;
-
-import net.hydromatic.linq4j.Ord;
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.sql.SqlCallBinding;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperandCountRange;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
 
+import java.util.AbstractList;
+import java.util.List;
+
 /**
  * This class allows multiple existing {@link SqlOperandTypeChecker} rules to be
  * combined into one rule. For example, allowing an operand to be either string
@@ -76,6 +79,7 @@ public class CompositeOperandTypeChecker
     implements SqlSingleOperandTypeChecker {
   //~ Enums ------------------------------------------------------------------
 
+  /** How operands are composed. */
   public enum Composition {
     AND, OR, SEQUENCE
   }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/type/CursorReturnTypeInference.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/CursorReturnTypeInference.java b/core/src/main/java/org/apache/calcite/sql/type/CursorReturnTypeInference.java
index bb4f9cf..117283d 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/CursorReturnTypeInference.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/CursorReturnTypeInference.java
@@ -14,10 +14,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.type;
+package org.apache.calcite.sql.type;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlOperatorBinding;
 
 /**
  * Returns the rowtype of a cursor of the operand at a particular 0-based

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/type/ExplicitOperandTypeInference.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/ExplicitOperandTypeInference.java b/core/src/main/java/org/apache/calcite/sql/type/ExplicitOperandTypeInference.java
index 8eeef4b..684388e 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/ExplicitOperandTypeInference.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/ExplicitOperandTypeInference.java
@@ -14,10 +14,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.type;
+package org.apache.calcite.sql.type;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlCallBinding;
 
 import com.google.common.collect.ImmutableList;
 
@@ -32,7 +32,8 @@ public class ExplicitOperandTypeInference implements SqlOperandTypeInference {
 
   //~ Constructors -----------------------------------------------------------
 
-  /** Use {@link org.eigenbase.sql.type.InferTypes#explicit(java.util.List)}. */
+  /** Use
+   * {@link org.apache.calcite.sql.type.InferTypes#explicit(java.util.List)}. */
   ExplicitOperandTypeInference(ImmutableList<RelDataType> paramTypes) {
     this.paramTypes = paramTypes;
   }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/type/ExplicitReturnTypeInference.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/ExplicitReturnTypeInference.java b/core/src/main/java/org/apache/calcite/sql/type/ExplicitReturnTypeInference.java
index c3c0117..7d891ec 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/ExplicitReturnTypeInference.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/ExplicitReturnTypeInference.java
@@ -14,10 +14,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.type;
+package org.apache.calcite.sql.type;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelProtoDataType;
+import org.apache.calcite.sql.SqlOperatorBinding;
 
 /**
  * A {@link SqlReturnTypeInference} which always returns the same SQL type.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/type/ExtraSqlTypes.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/ExtraSqlTypes.java b/core/src/main/java/org/apache/calcite/sql/type/ExtraSqlTypes.java
index 2664f85..1774625 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/ExtraSqlTypes.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/ExtraSqlTypes.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.type;
+package org.apache.calcite.sql.type;
 
 /**
  * Holds constants associated with SQL types introduced after the earliest

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/type/FamilyOperandTypeChecker.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/FamilyOperandTypeChecker.java b/core/src/main/java/org/apache/calcite/sql/type/FamilyOperandTypeChecker.java
index 109529c..c421727 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/FamilyOperandTypeChecker.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/FamilyOperandTypeChecker.java
@@ -14,18 +14,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.type;
+package org.apache.calcite.sql.type;
 
-import java.util.*;
-
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
-
-import net.hydromatic.linq4j.Ord;
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlCallBinding;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperandCountRange;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlUtil;
 
 import com.google.common.collect.ImmutableList;
 
-import static org.eigenbase.util.Static.RESOURCE;
+import java.util.List;
+
+import static org.apache.calcite.util.Static.RESOURCE;
 
 /**
  * Operand type-checking strategy which checks operands for inclusion in type

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/type/InferTypes.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/InferTypes.java b/core/src/main/java/org/apache/calcite/sql/type/InferTypes.java
index d9d56f9..f4ab473 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/InferTypes.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/InferTypes.java
@@ -14,22 +14,22 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.type;
+package org.apache.calcite.sql.type;
 
-import java.util.List;
-
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.reltype.RelDataTypeFactory;
-import org.eigenbase.sql.SqlCallBinding;
-import org.eigenbase.sql.SqlNode;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.sql.SqlCallBinding;
+import org.apache.calcite.sql.SqlNode;
 
 import com.google.common.collect.ImmutableList;
 
+import java.util.List;
+
 /**
  * Strategies for inferring operand types.
  *
- * @see org.eigenbase.sql.type.SqlOperandTypeInference
- * @see org.eigenbase.sql.type.ReturnTypes
+ * @see org.apache.calcite.sql.type.SqlOperandTypeInference
+ * @see org.apache.calcite.sql.type.ReturnTypes
  */
 public abstract class InferTypes {
   private InferTypes() {}

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/type/IntervalSqlType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/IntervalSqlType.java b/core/src/main/java/org/apache/calcite/sql/type/IntervalSqlType.java
index c82671c..73188b4 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/IntervalSqlType.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/IntervalSqlType.java
@@ -14,13 +14,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.type;
+package org.apache.calcite.sql.type;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.sql.pretty.SqlPrettyWriter;
-import org.eigenbase.sql.util.SqlString;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactoryImpl;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.sql.SqlDialect;
+import org.apache.calcite.sql.SqlIntervalQualifier;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.pretty.SqlPrettyWriter;
+import org.apache.calcite.sql.util.SqlString;
 
 /**
  * IntervalSqlType represents a standard SQL datetime interval type.
@@ -145,8 +148,7 @@ public class IntervalSqlType extends AbstractSqlType {
     return intervalQualifier.getStartPrecision(typeSystem);
   }
 
-  @Override
-  public int getScale() {
+  @Override public int getScale() {
     return intervalQualifier.getFractionalSecondPrecision(typeSystem);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/type/JavaToSqlTypeConversionRules.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/JavaToSqlTypeConversionRules.java b/core/src/main/java/org/apache/calcite/sql/type/JavaToSqlTypeConversionRules.java
index 0dfbddc..dd5097d 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/JavaToSqlTypeConversionRules.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/JavaToSqlTypeConversionRules.java
@@ -14,15 +14,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.type;
-
-import java.math.*;
-import java.sql.*;
-import java.sql.Date;
-import java.util.*;
+package org.apache.calcite.sql.type;
 
 import com.google.common.collect.ImmutableMap;
 
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.ResultSet;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.util.List;
+import java.util.Map;
+
 /**
  * JavaToSqlTypeConversionRules defines mappings from common Java types to
  * corresponding SQL types.
@@ -72,8 +75,9 @@ public class JavaToSqlTypeConversionRules {
   //~ Methods ----------------------------------------------------------------
 
   /**
-   * Returns the {@link org.eigenbase.util.Glossary#SINGLETON_PATTERN
-   * singleton} instance.
+   * Returns the
+   * {@link org.apache.calcite.util.Glossary#SINGLETON_PATTERN singleton}
+   * instance.
    */
   public static JavaToSqlTypeConversionRules instance() {
     return INSTANCE;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/type/LiteralOperandTypeChecker.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/LiteralOperandTypeChecker.java b/core/src/main/java/org/apache/calcite/sql/type/LiteralOperandTypeChecker.java
index 4d50598..83161d1 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/LiteralOperandTypeChecker.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/LiteralOperandTypeChecker.java
@@ -14,12 +14,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.type;
+package org.apache.calcite.sql.type;
 
-import org.eigenbase.sql.*;
-import org.eigenbase.util.*;
+import org.apache.calcite.sql.SqlCallBinding;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperandCountRange;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlUtil;
+import org.apache.calcite.util.Util;
 
-import static org.eigenbase.util.Static.RESOURCE;
+import static org.apache.calcite.util.Static.RESOURCE;
 
 /**
  * Parameter type-checking strategy type must be a literal (whether null is

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/type/MapSqlType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/MapSqlType.java b/core/src/main/java/org/apache/calcite/sql/type/MapSqlType.java
index 8a8e4f8..61f8b79 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/MapSqlType.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/MapSqlType.java
@@ -14,10 +14,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.type;
+package org.apache.calcite.sql.type;
 
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.reltype.RelDataTypeFamily;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFamily;
 
 /**
  * SQL map type.
@@ -46,13 +46,11 @@ public class MapSqlType extends AbstractSqlType {
 
   //~ Methods ----------------------------------------------------------------
 
-  @Override
-  public RelDataType getValueType() {
+  @Override public RelDataType getValueType() {
     return valueType;
   }
 
-  @Override
-  public RelDataType getKeyType() {
+  @Override public RelDataType getKeyType() {
     return keyType;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/type/MatchReturnTypeInference.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/MatchReturnTypeInference.java b/core/src/main/java/org/apache/calcite/sql/type/MatchReturnTypeInference.java
index 99d3533..5c60187 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/MatchReturnTypeInference.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/MatchReturnTypeInference.java
@@ -14,15 +14,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.type;
+package org.apache.calcite.sql.type;
 
-import java.util.List;
-
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlOperatorBinding;
 
 import com.google.common.collect.ImmutableList;
 
+import java.util.List;
+
 /**
  * Returns the first type that matches a set of given {@link SqlTypeName}s. If
  * no match could be found, null is returned.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/type/MultisetOperandTypeChecker.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/MultisetOperandTypeChecker.java b/core/src/main/java/org/apache/calcite/sql/type/MultisetOperandTypeChecker.java
index ae78142..5b33fce 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/MultisetOperandTypeChecker.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/MultisetOperandTypeChecker.java
@@ -14,14 +14,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.type;
+package org.apache.calcite.sql.type;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlCallBinding;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperandCountRange;
+import org.apache.calcite.sql.SqlOperator;
 
 import com.google.common.collect.ImmutableList;
 
-import static org.eigenbase.util.Static.RESOURCE;
+import static org.apache.calcite.util.Static.RESOURCE;
 
 /**
  * Parameter type-checking strategy types must be [nullable] Multiset,

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/type/MultisetSqlType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/MultisetSqlType.java b/core/src/main/java/org/apache/calcite/sql/type/MultisetSqlType.java
index 0d30b4b..eae3d62 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/MultisetSqlType.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/MultisetSqlType.java
@@ -14,10 +14,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.type;
+package org.apache.calcite.sql.type;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.util.*;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFamily;
 
 /**
  * MultisetSqlType represents a standard SQL2003 multiset type.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/type/ObjectSqlType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/ObjectSqlType.java b/core/src/main/java/org/apache/calcite/sql/type/ObjectSqlType.java
index 78d3cf1..dac49f2 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/ObjectSqlType.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/ObjectSqlType.java
@@ -14,12 +14,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.type;
+package org.apache.calcite.sql.type;
 
-import java.util.List;
+import org.apache.calcite.rel.type.RelDataTypeComparability;
+import org.apache.calcite.rel.type.RelDataTypeFamily;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.sql.SqlIdentifier;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
+import java.util.List;
 
 /**
  * ObjectSqlType represents an SQL structured user-defined type.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/type/OperandTypes.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/OperandTypes.java b/core/src/main/java/org/apache/calcite/sql/type/OperandTypes.java
index 7e7572e..2466351 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/OperandTypes.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/OperandTypes.java
@@ -14,23 +14,29 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.type;
+package org.apache.calcite.sql.type;
 
-import java.util.*;
-
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeComparability;
+import org.apache.calcite.sql.SqlCallBinding;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperandCountRange;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlUtil;
 
 import com.google.common.collect.ImmutableList;
 
-import static org.eigenbase.util.Static.RESOURCE;
+import java.util.List;
+
+import static org.apache.calcite.util.Static.RESOURCE;
 
 /**
  * Strategies for checking operand types.
  *
  * <p>This class defines singleton instances of strategy objects for operand
- * type checking. {@link org.eigenbase.sql.type.ReturnTypes}
- * and {@link org.eigenbase.sql.type.InferTypes} provide similar strategies
+ * type checking. {@link org.apache.calcite.sql.type.ReturnTypes}
+ * and {@link org.apache.calcite.sql.type.InferTypes} provide similar strategies
  * for operand type inference and operator return type inference.
  *
  * <p>Note to developers: avoid anonymous inner classes here except for unique,
@@ -38,9 +44,9 @@ import static org.eigenbase.util.Static.RESOURCE;
  * class. If you find yourself copying and pasting an existing strategy's
  * anonymous inner class, you're making a mistake.
  *
- * @see org.eigenbase.sql.type.SqlOperandTypeChecker
- * @see org.eigenbase.sql.type.ReturnTypes
- * @see org.eigenbase.sql.type.InferTypes
+ * @see org.apache.calcite.sql.type.SqlOperandTypeChecker
+ * @see org.apache.calcite.sql.type.ReturnTypes
+ * @see org.apache.calcite.sql.type.InferTypes
  */
 public abstract class OperandTypes {
   private OperandTypes() {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/type/OperandsTypeChecking.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/OperandsTypeChecking.java b/core/src/main/java/org/apache/calcite/sql/type/OperandsTypeChecking.java
index 4a9bafb..ebc01f5 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/OperandsTypeChecking.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/OperandsTypeChecking.java
@@ -14,10 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.type;
-
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
+package org.apache.calcite.sql.type;
 
 /**
  * Strategies to check for allowed operand types of an operator call.


[08/58] [abbrv] [partial] incubator-calcite git commit: [CALCITE-306] Standardize code style for "import package.*; "

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/resources/org/apache/calcite/test/HepPlannerTest.xml
----------------------------------------------------------------------
diff --git a/core/src/test/resources/org/apache/calcite/test/HepPlannerTest.xml b/core/src/test/resources/org/apache/calcite/test/HepPlannerTest.xml
index bf758ad..217eb36 100644
--- a/core/src/test/resources/org/apache/calcite/test/HepPlannerTest.xml
+++ b/core/src/test/resources/org/apache/calcite/test/HepPlannerTest.xml
@@ -22,20 +22,20 @@ limitations under the License.
     </Resource>
     <Resource name="planBefore">
       <![CDATA[
-ProjectRel(DEPTNO=[$0])
-  JoinRel(condition=[true], joinType=[inner])
-    ProjectRel(DEPTNO=[$0], NAME=[$1])
-      TableAccessRel(table=[[CATALOG, SALES, DEPT]])
-    ProjectRel(DEPTNO=[$0], NAME=[$1])
-      TableAccessRel(table=[[CATALOG, SALES, DEPT]])
+LogicalProject(DEPTNO=[$0])
+  LogicalJoin(condition=[true], joinType=[inner])
+    LogicalProject(DEPTNO=[$0], NAME=[$1])
+      LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+    LogicalProject(DEPTNO=[$0], NAME=[$1])
+      LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
     </Resource>
     <Resource name="planAfter">
       <![CDATA[
-ProjectRel(DEPTNO=[$0])
-  JoinRel(condition=[true], joinType=[inner])
-    TableAccessRel(table=[[CATALOG, SALES, DEPT]])
-    TableAccessRel(table=[[CATALOG, SALES, DEPT]])
+LogicalProject(DEPTNO=[$0])
+  LogicalJoin(condition=[true], joinType=[inner])
+    LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+    LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
     </Resource>
   </TestCase>
@@ -45,28 +45,28 @@ ProjectRel(DEPTNO=[$0])
     </Resource>
     <Resource name="planBefore">
       <![CDATA[
-UnionRel(all=[false])
-  UnionRel(all=[false])
-    ProjectRel(NAME=[$1])
-      TableAccessRel(table=[[CATALOG, SALES, DEPT]])
-    ProjectRel(ENAME=[$1])
-      TableAccessRel(table=[[CATALOG, SALES, EMP]])
-  ProjectRel(ENAME=[$0])
-    TableAccessRel(table=[[CATALOG, SALES, BONUS]])
+LogicalUnion(all=[false])
+  LogicalUnion(all=[false])
+    LogicalProject(NAME=[$1])
+      LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+    LogicalProject(ENAME=[$1])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+  LogicalProject(ENAME=[$0])
+    LogicalTableScan(table=[[CATALOG, SALES, BONUS]])
 ]]>
     </Resource>
     <Resource name="planAfter">
       <![CDATA[
-AggregateRel(group=[{0}])
-  UnionRel(all=[true])
-    AggregateRel(group=[{0}])
-      UnionRel(all=[true])
-        ProjectRel(NAME=[$1])
-          TableAccessRel(table=[[CATALOG, SALES, DEPT]])
-        ProjectRel(ENAME=[$1])
-          TableAccessRel(table=[[CATALOG, SALES, EMP]])
-    ProjectRel(ENAME=[$0])
-      TableAccessRel(table=[[CATALOG, SALES, BONUS]])
+LogicalAggregate(group=[{0}])
+  LogicalUnion(all=[true])
+    LogicalAggregate(group=[{0}])
+      LogicalUnion(all=[true])
+        LogicalProject(NAME=[$1])
+          LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+        LogicalProject(ENAME=[$1])
+          LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalProject(ENAME=[$0])
+      LogicalTableScan(table=[[CATALOG, SALES, BONUS]])
 ]]>
     </Resource>
   </TestCase>
@@ -76,27 +76,27 @@ AggregateRel(group=[{0}])
     </Resource>
     <Resource name="planBefore">
       <![CDATA[
-UnionRel(all=[false])
-  UnionRel(all=[false])
-    ProjectRel(NAME=[$1])
-      TableAccessRel(table=[[CATALOG, SALES, DEPT]])
-    ProjectRel(ENAME=[$1])
-      TableAccessRel(table=[[CATALOG, SALES, EMP]])
-  ProjectRel(ENAME=[$0])
-    TableAccessRel(table=[[CATALOG, SALES, BONUS]])
+LogicalUnion(all=[false])
+  LogicalUnion(all=[false])
+    LogicalProject(NAME=[$1])
+      LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+    LogicalProject(ENAME=[$1])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+  LogicalProject(ENAME=[$0])
+    LogicalTableScan(table=[[CATALOG, SALES, BONUS]])
 ]]>
     </Resource>
     <Resource name="planAfter">
       <![CDATA[
-AggregateRel(group=[{0}])
-  UnionRel(all=[true])
-    UnionRel(all=[false])
-      ProjectRel(NAME=[$1])
-        TableAccessRel(table=[[CATALOG, SALES, DEPT]])
-      ProjectRel(ENAME=[$1])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
-    ProjectRel(ENAME=[$0])
-      TableAccessRel(table=[[CATALOG, SALES, BONUS]])
+LogicalAggregate(group=[{0}])
+  LogicalUnion(all=[true])
+    LogicalUnion(all=[false])
+      LogicalProject(NAME=[$1])
+        LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+      LogicalProject(ENAME=[$1])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalProject(ENAME=[$0])
+      LogicalTableScan(table=[[CATALOG, SALES, BONUS]])
 ]]>
     </Resource>
   </TestCase>
@@ -106,27 +106,27 @@ AggregateRel(group=[{0}])
     </Resource>
     <Resource name="planBefore">
       <![CDATA[
-UnionRel(all=[false])
-  UnionRel(all=[false])
-    ProjectRel(NAME=[$1])
-      TableAccessRel(table=[[CATALOG, SALES, DEPT]])
-    ProjectRel(ENAME=[$1])
-      TableAccessRel(table=[[CATALOG, SALES, EMP]])
-  ProjectRel(ENAME=[$0])
-    TableAccessRel(table=[[CATALOG, SALES, BONUS]])
+LogicalUnion(all=[false])
+  LogicalUnion(all=[false])
+    LogicalProject(NAME=[$1])
+      LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+    LogicalProject(ENAME=[$1])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+  LogicalProject(ENAME=[$0])
+    LogicalTableScan(table=[[CATALOG, SALES, BONUS]])
 ]]>
     </Resource>
     <Resource name="planAfter">
       <![CDATA[
-UnionRel(all=[false])
-  AggregateRel(group=[{0}])
-    UnionRel(all=[true])
-      ProjectRel(NAME=[$1])
-        TableAccessRel(table=[[CATALOG, SALES, DEPT]])
-      ProjectRel(ENAME=[$1])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
-  ProjectRel(ENAME=[$0])
-    TableAccessRel(table=[[CATALOG, SALES, BONUS]])
+LogicalUnion(all=[false])
+  LogicalAggregate(group=[{0}])
+    LogicalUnion(all=[true])
+      LogicalProject(NAME=[$1])
+        LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+      LogicalProject(ENAME=[$1])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+  LogicalProject(ENAME=[$0])
+    LogicalTableScan(table=[[CATALOG, SALES, BONUS]])
 ]]>
     </Resource>
   </TestCase>
@@ -136,28 +136,28 @@ UnionRel(all=[false])
     </Resource>
     <Resource name="planBefore">
       <![CDATA[
-IntersectRel(all=[false])
-  UnionRel(all=[false])
-    ProjectRel(NAME=[$1])
-      TableAccessRel(table=[[CATALOG, SALES, DEPT]])
-    ProjectRel(ENAME=[$1])
-      TableAccessRel(table=[[CATALOG, SALES, EMP]])
-  ProjectRel(FNAME=[$1])
-    TableAccessRel(table=[[CATALOG, CUSTOMER, CONTACT]])
+LogicalIntersect(all=[false])
+  LogicalUnion(all=[false])
+    LogicalProject(NAME=[$1])
+      LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+    LogicalProject(ENAME=[$1])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+  LogicalProject(FNAME=[$1])
+    LogicalTableScan(table=[[CATALOG, CUSTOMER, CONTACT]])
 ]]>
     </Resource>
     <Resource name="planAfter">
       <![CDATA[
-IntersectRel(all=[false])
-  UnionRel(all=[false])
-    ProjectRel(NAME=[CAST($0):VARCHAR(20) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary" NOT NULL])
-      ProjectRel(NAME=[$1])
-        TableAccessRel(table=[[CATALOG, SALES, DEPT]])
-    ProjectRel(ENAME=[$1])
-      TableAccessRel(table=[[CATALOG, SALES, EMP]])
-  ProjectRel(FNAME=[CAST($0):VARCHAR(20) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary" NOT NULL])
-    ProjectRel(FNAME=[$1])
-      TableAccessRel(table=[[CATALOG, CUSTOMER, CONTACT]])
+LogicalIntersect(all=[false])
+  LogicalUnion(all=[false])
+    LogicalProject(NAME=[CAST($0):VARCHAR(20) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary" NOT NULL])
+      LogicalProject(NAME=[$1])
+        LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+    LogicalProject(ENAME=[$1])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+  LogicalProject(FNAME=[CAST($0):VARCHAR(20) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary" NOT NULL])
+    LogicalProject(FNAME=[$1])
+      LogicalTableScan(table=[[CATALOG, CUSTOMER, CONTACT]])
 ]]>
     </Resource>
   </TestCase>
@@ -167,15 +167,15 @@ IntersectRel(all=[false])
     </Resource>
     <Resource name="planBefore">
       <![CDATA[
-ProjectRel(EXPR$0=[UPPER($0)])
-  ProjectRel(ENAME=[LOWER($1)])
-    TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(EXPR$0=[UPPER($0)])
+  LogicalProject(ENAME=[LOWER($1)])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
     </Resource>
     <Resource name="planAfter">
       <![CDATA[
-CalcRel(expr#0..8=[{inputs}], expr#9=[LOWER($t1)], expr#10=[UPPER($t9)], EXPR$0=[$t10])
-  TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalCalc(expr#0..8=[{inputs}], expr#9=[LOWER($t1)], expr#10=[UPPER($t9)], EXPR$0=[$t10])
+  LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
     </Resource>
   </TestCase>
@@ -185,16 +185,16 @@ CalcRel(expr#0..8=[{inputs}], expr#9=[LOWER($t1)], expr#10=[UPPER($t9)], EXPR$0=
     </Resource>
     <Resource name="planBefore">
       <![CDATA[
-ProjectRel(NAME=[$1])
-  FilterRel(condition=[=($0, 12)])
-    TableAccessRel(table=[[CATALOG, SALES, DEPT]])
+LogicalProject(NAME=[$1])
+  LogicalFilter(condition=[=($0, 12)])
+    LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
     </Resource>
     <Resource name="planAfter">
       <![CDATA[
-ProjectRel(NAME=[$1])
-  CalcRel(expr#0..1=[{inputs}], expr#2=[12], expr#3=[=($t0, $t2)], proj#0..1=[{exprs}], $condition=[$t3])
-    TableAccessRel(table=[[CATALOG, SALES, DEPT]])
+LogicalProject(NAME=[$1])
+  LogicalCalc(expr#0..1=[{inputs}], expr#2=[12], expr#3=[=($t0, $t2)], proj#0..1=[{exprs}], $condition=[$t3])
+    LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
     </Resource>
   </TestCase>
@@ -204,15 +204,15 @@ ProjectRel(NAME=[$1])
     </Resource>
     <Resource name="planBefore">
       <![CDATA[
-ProjectRel(EXPR$0=[UPPER($1)])
-  FilterRel(condition=[=($0, 20)])
-    TableAccessRel(table=[[CATALOG, SALES, DEPT]])
+LogicalProject(EXPR$0=[UPPER($1)])
+  LogicalFilter(condition=[=($0, 20)])
+    LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
     </Resource>
     <Resource name="planAfter">
       <![CDATA[
-CalcRel(expr#0..1=[{inputs}], expr#2=[UPPER($t1)], expr#3=[20], expr#4=[=($t0, $t3)], EXPR$0=[$t2], $condition=[$t4])
-  TableAccessRel(table=[[CATALOG, SALES, DEPT]])
+LogicalCalc(expr#0..1=[{inputs}], expr#2=[UPPER($t1)], expr#3=[20], expr#4=[=($t0, $t3)], EXPR$0=[$t2], $condition=[$t4])
+  LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
     </Resource>
   </TestCase>


[26/58] [abbrv] [partial] incubator-calcite git commit: [CALCITE-306] Standardize code style for "import package.*; "

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/SqlUpdate.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlUpdate.java b/core/src/main/java/org/apache/calcite/sql/SqlUpdate.java
index 173571e..6773aa8 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlUpdate.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlUpdate.java
@@ -14,14 +14,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
+package org.apache.calcite.sql;
 
-import java.util.List;
+import org.apache.calcite.sql.parser.SqlParserPos;
+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.ImmutableNullableList;
+import org.apache.calcite.util.Pair;
 
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.sql.validate.*;
-import org.eigenbase.util.ImmutableNullableList;
-import org.eigenbase.util.Pair;
+import java.util.List;
 
 /**
  * A <code>SqlUpdate</code> is a node of a parse tree which represents an UPDATE
@@ -72,8 +74,7 @@ public class SqlUpdate extends SqlCall {
         sourceExpressionList, condition, sourceSelect, alias);
   }
 
-  @Override
-  public void setOperand(int i, SqlNode operand) {
+  @Override public void setOperand(int i, SqlNode operand) {
     switch (i) {
     case 0:
       targetTable = (SqlIdentifier) operand;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/SqlUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlUtil.java b/core/src/main/java/org/apache/calcite/sql/SqlUtil.java
index dadc3a3..e2430e7 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlUtil.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlUtil.java
@@ -14,24 +14,36 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
-
-import java.nio.charset.*;
-import java.sql.*;
-import java.text.*;
-import java.util.*;
-
-import org.eigenbase.reltype.*;
-import org.eigenbase.resource.Resources;
-import org.eigenbase.sql.fun.*;
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.util.*;
-import org.eigenbase.util14.*;
+package org.apache.calcite.sql;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypePrecedenceList;
+import org.apache.calcite.runtime.CalciteContextException;
+import org.apache.calcite.runtime.CalciteException;
+import org.apache.calcite.runtime.Resources;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.SqlTypeFamily;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.type.SqlTypeUtil;
+import org.apache.calcite.util.BarfingInvocationHandler;
+import org.apache.calcite.util.ConversionUtil;
+import org.apache.calcite.util.NlsString;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
 
 import com.google.common.collect.Lists;
 
-import static org.eigenbase.util.Static.RESOURCE;
+import java.nio.charset.Charset;
+import java.sql.DatabaseMetaData;
+import java.sql.SQLException;
+import java.text.MessageFormat;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import static org.apache.calcite.util.Static.RESOURCE;
 
 /**
  * Contains utility functions related to SQL parsing, all static.
@@ -637,11 +649,11 @@ public abstract class SqlUtil {
   /**
    * Wraps an exception with context.
    */
-  public static EigenbaseException newContextException(
+  public static CalciteException newContextException(
       final SqlParserPos pos,
       Resources.ExInst<?> e,
       String inputText) {
-    EigenbaseContextException ex = newContextException(pos, e);
+    CalciteContextException ex = newContextException(pos, e);
     ex.setOriginalStatement(inputText);
     return ex;
   }
@@ -649,7 +661,7 @@ public abstract class SqlUtil {
   /**
    * Wraps an exception with context.
    */
-  public static EigenbaseContextException newContextException(
+  public static CalciteContextException newContextException(
       final SqlParserPos pos,
       Resources.ExInst<?> e) {
     int line = pos.getLineNum();
@@ -662,13 +674,13 @@ public abstract class SqlUtil {
   /**
    * Wraps an exception with context.
    */
-  public static EigenbaseContextException newContextException(
+  public static CalciteContextException newContextException(
       int line,
       int col,
       int endLine,
       int endCol,
       Resources.ExInst<?> e) {
-    EigenbaseContextException contextExcn =
+    CalciteContextException contextExcn =
         (line == endLine && col == endCol
             ? RESOURCE.validatorContextPoint(line, col)
             : RESOURCE.validatorContext(line, col, endLine, endCol)).ex(e.ex());
@@ -686,10 +698,11 @@ public abstract class SqlUtil {
   }
 
   /**
-   * Creates the type of an {@link NlsString}.
+   * Creates the type of an {@link org.apache.calcite.util.NlsString}.
    *
-   * <p>The type inherits the The NlsString's {@link Charset} and {@link
-   * SqlCollation}, if they are set, otherwise it gets the system defaults.
+   * <p>The type inherits the The NlsString's {@link Charset} and
+   * {@link SqlCollation}, if they are set, otherwise it gets the system
+   * defaults.
    *
    * @param typeFactory Type factory
    * @param str         String
@@ -755,8 +768,9 @@ public abstract class SqlUtil {
 
   /**
    * Handles particular {@link DatabaseMetaData} methods; invocations of other
-   * methods will fall through to the base class, {@link
-   * BarfingInvocationHandler}, which will throw an error.
+   * methods will fall through to the base class,
+   * {@link org.apache.calcite.util.BarfingInvocationHandler}, which will throw
+   * an error.
    */
   public static class DatabaseMetaDataInvocationHandler
       extends BarfingInvocationHandler {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/SqlValuesOperator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlValuesOperator.java b/core/src/main/java/org/apache/calcite/sql/SqlValuesOperator.java
index 3ad8195..ef6ae60 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlValuesOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlValuesOperator.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
+package org.apache.calcite.sql;
 
 /**
  * The <code>VALUES</code> operator.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 42eab28..be9d9fc 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlWindow.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlWindow.java
@@ -14,25 +14,29 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
-
-import java.util.*;
-
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.rex.RexWindowBound;
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.sql.type.ReturnTypes;
-import org.eigenbase.sql.type.SqlTypeFamily;
-import org.eigenbase.sql.util.SqlBasicVisitor;
-import org.eigenbase.sql.util.SqlVisitor;
-import org.eigenbase.sql.validate.*;
-import org.eigenbase.util.*;
-
-import net.hydromatic.linq4j.Ord;
+package org.apache.calcite.sql;
+
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexWindowBound;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.ReturnTypes;
+import org.apache.calcite.sql.type.SqlTypeFamily;
+import org.apache.calcite.sql.util.SqlBasicVisitor;
+import org.apache.calcite.sql.util.SqlVisitor;
+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.ImmutableNullableList;
+import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
 
-import static org.eigenbase.util.Static.RESOURCE;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.calcite.util.Static.RESOURCE;
 
 /**
  * SQL window specification.
@@ -218,10 +222,12 @@ public class SqlWindow extends SqlCall {
    * Returns if the window is guaranteed to have rows.
    * This is useful to refine data type of window aggregates.
    * For instance sum(non-nullable) over (empty window) is NULL.
+   *
    * @return true when the window is non-empty
-   * @see org.eigenbase.rel.WindowRelBase.Window#isAlwaysNonEmpty()
+   *
+   * @see org.apache.calcite.rel.core.Window.Group#isAlwaysNonEmpty()
    * @see SqlOperatorBinding#getGroupCount()
-   * @see org.eigenbase.sql.validate.SqlValidatorImpl#resolveWindow(SqlNode, org.eigenbase.sql.validate.SqlValidatorScope, boolean)
+   * @see org.apache.calcite.sql.validate.SqlValidatorImpl#resolveWindow(SqlNode, org.apache.calcite.sql.validate.SqlValidatorScope, boolean)
    */
   public boolean isAlwaysNonEmpty() {
     final SqlWindow tmp;
@@ -517,8 +523,8 @@ public class SqlWindow extends SqlCall {
         || allowPartial.booleanValue();
   }
 
-  @Override
-  public void validate(SqlValidator validator, SqlValidatorScope scope) {
+  @Override public void validate(SqlValidator validator,
+      SqlValidatorScope scope) {
     SqlValidatorScope operandScope = scope; // REVIEW
 
     SqlIdentifier declName = this.declName;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/SqlWith.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlWith.java b/core/src/main/java/org/apache/calcite/sql/SqlWith.java
index fd4b640..4e772ac 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlWith.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlWith.java
@@ -14,16 +14,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
+package org.apache.calcite.sql;
 
-import java.util.List;
-
-import org.eigenbase.sql.parser.SqlParserPos;
-import org.eigenbase.sql.validate.SqlValidator;
-import org.eigenbase.sql.validate.SqlValidatorScope;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql.validate.SqlValidatorScope;
 
 import com.google.common.collect.ImmutableList;
 
+import java.util.List;
+
 /**
  * The WITH clause of a query. It wraps a SELECT, UNION, or INTERSECT.
  */
@@ -45,8 +45,7 @@ public class SqlWith extends SqlCall {
     return SqlKind.WITH;
   }
 
-  @Override
-  public SqlOperator getOperator() {
+  @Override public SqlOperator getOperator() {
     return SqlWithOperator.INSTANCE;
   }
 
@@ -54,8 +53,8 @@ public class SqlWith extends SqlCall {
     return ImmutableList.of(withList, body);
   }
 
-  @Override
-  public void validate(SqlValidator validator, SqlValidatorScope scope) {
+  @Override public void validate(SqlValidator validator,
+      SqlValidatorScope scope) {
     validator.validateWith(this, scope);
   }
 
@@ -92,9 +91,8 @@ public class SqlWith extends SqlCall {
     }
 
 
-    @Override
-    public SqlCall createCall(SqlLiteral functionQualifier, SqlParserPos pos,
-        SqlNode... operands) {
+    @Override public SqlCall createCall(SqlLiteral functionQualifier,
+        SqlParserPos pos, SqlNode... operands) {
       return new SqlWith(pos, (SqlNodeList) operands[0], operands[1]);
     }
 
@@ -107,4 +105,4 @@ public class SqlWith extends SqlCall {
   }
 }
 
-// End SqlWithOperator.java
+// End SqlWith.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/SqlWithItem.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlWithItem.java b/core/src/main/java/org/apache/calcite/sql/SqlWithItem.java
index 35e74d0..7fe5c62 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlWithItem.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlWithItem.java
@@ -14,12 +14,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
+package org.apache.calcite.sql;
 
-import java.util.List;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.util.ImmutableNullableList;
 
-import org.eigenbase.sql.parser.SqlParserPos;
-import org.eigenbase.util.ImmutableNullableList;
+import java.util.List;
 
 /**
  * An item in a WITH clause of a query.
@@ -48,8 +48,7 @@ public class SqlWithItem extends SqlCall {
     return ImmutableNullableList.of(name, columnList, query);
   }
 
-  @Override
-  public void setOperand(int i, SqlNode operand) {
+  @Override public void setOperand(int i, SqlNode operand) {
     switch (i) {
     case 0:
       name = (SqlIdentifier) operand;
@@ -97,9 +96,8 @@ public class SqlWithItem extends SqlCall {
       withItem.query.unparse(writer, getLeftPrec(), getRightPrec());
     }
 
-    @Override
-    public SqlCall createCall(SqlLiteral functionQualifier, SqlParserPos pos,
-        SqlNode... operands) {
+    @Override public SqlCall createCall(SqlLiteral functionQualifier,
+        SqlParserPos pos, SqlNode... operands) {
       assert functionQualifier == null;
       assert operands.length == 3;
       return new SqlWithItem(pos, (SqlIdentifier) operands[0],
@@ -108,4 +106,4 @@ public class SqlWithItem extends SqlCall {
   }
 }
 
-// End SqlWithItemOperator.java
+// End SqlWithItem.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/SqlWriter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlWriter.java b/core/src/main/java/org/apache/calcite/sql/SqlWriter.java
index a5f01ef..ac03856 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlWriter.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlWriter.java
@@ -14,9 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
+package org.apache.calcite.sql;
 
-import org.eigenbase.sql.util.SqlString;
+import org.apache.calcite.sql.util.SqlString;
 
 /**
  * A <code>SqlWriter</code> is the target to construct a SQL statement from a
@@ -451,14 +451,15 @@ public interface SqlWriter {
    * are empty strings, but it consists of a sequence of clauses. "SELECT",
    * "FROM", "WHERE" are separators.
    *
-   * <p>A frame is current between a call to one of the {@link
-   * SqlWriter#startList} methods and the call to {@link
-   * SqlWriter#endList(Frame)}. If other code starts a frame in the mean time,
-   * the sub-frame is put onto a stack.
+   * <p>A frame is current between a call to one of the
+   * {@link SqlWriter#startList} methods and the call to
+   * {@link SqlWriter#endList(Frame)}. If other code starts a frame in the mean
+   * time, the sub-frame is put onto a stack.
    */
   public interface Frame {
   }
 
+  /** Frame type. */
   interface FrameType {
     /**
      * Returns the name of this frame type.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/advise/SqlAdvisor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/advise/SqlAdvisor.java b/core/src/main/java/org/apache/calcite/sql/advise/SqlAdvisor.java
index 7643b00..818d200 100644
--- a/core/src/main/java/org/apache/calcite/sql/advise/SqlAdvisor.java
+++ b/core/src/main/java/org/apache/calcite/sql/advise/SqlAdvisor.java
@@ -14,16 +14,29 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.advise;
-
-import java.util.*;
-import java.util.logging.*;
-
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.sql.validate.*;
-import org.eigenbase.trace.*;
-import org.eigenbase.util.*;
+package org.apache.calcite.sql.advise;
+
+import org.apache.calcite.runtime.CalciteContextException;
+import org.apache.calcite.runtime.CalciteException;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.parser.SqlAbstractParserImpl;
+import org.apache.calcite.sql.parser.SqlParseException;
+import org.apache.calcite.sql.parser.SqlParser;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.validate.SqlMoniker;
+import org.apache.calcite.sql.validate.SqlMonikerImpl;
+import org.apache.calcite.sql.validate.SqlMonikerType;
+import org.apache.calcite.sql.validate.SqlValidatorWithHints;
+import org.apache.calcite.util.Util;
+import org.apache.calcite.util.trace.CalciteTrace;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.logging.Logger;
 
 /**
  * An assistant which offers hints and corrections to a partially-formed SQL
@@ -32,7 +45,7 @@ import org.eigenbase.util.*;
 public class SqlAdvisor {
   //~ Static fields/initializers ---------------------------------------------
 
-  public static final Logger LOGGER = EigenbaseTrace.PARSER_LOGGER;
+  public static final Logger LOGGER = CalciteTrace.PARSER_LOGGER;
 
   //~ Instance fields --------------------------------------------------------
 
@@ -235,7 +248,7 @@ public class SqlAdvisor {
         }
       }
       return null;
-    } catch (EigenbaseException e) {
+    } catch (CalciteException e) {
       Util.swallow(e, null);
       return null;
     }
@@ -265,7 +278,7 @@ public class SqlAdvisor {
     SqlParserPos pos = new SqlParserPos(1, cursor + 1);
     try {
       return validator.lookupQualifiedName(sqlNode, pos);
-    } catch (EigenbaseContextException e) {
+    } catch (CalciteContextException e) {
       return null;
     } catch (java.lang.AssertionError e) {
       return null;
@@ -315,7 +328,7 @@ public class SqlAdvisor {
     }
     try {
       validator.validate(sqlNode);
-    } catch (EigenbaseContextException e) {
+    } catch (CalciteContextException e) {
       ValidateErrorInfo errInfo = new ValidateErrorInfo(e);
 
       // validator only returns 1 exception now
@@ -458,12 +471,12 @@ public class SqlAdvisor {
     }
 
     /**
-     * Creates a new ValidateErrorInfo with an EigenbaseContextException.
+     * Creates a new ValidateErrorInfo with an CalciteContextException.
      *
      * @param e Exception
      */
     public ValidateErrorInfo(
-        EigenbaseContextException e) {
+        CalciteContextException e) {
       this.startLineNum = e.getPosLine();
       this.startColumnNum = e.getPosColumn();
       this.endLineNum = e.getEndPosLine();

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/advise/SqlAdvisorGetHintsFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/advise/SqlAdvisorGetHintsFunction.java b/core/src/main/java/org/apache/calcite/sql/advise/SqlAdvisorGetHintsFunction.java
index b560b62..8e27d5d 100644
--- a/core/src/main/java/org/apache/calcite/sql/advise/SqlAdvisorGetHintsFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/advise/SqlAdvisorGetHintsFunction.java
@@ -14,7 +14,30 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.advise;
+package org.apache.calcite.sql.advise;
+
+import org.apache.calcite.DataContext;
+import org.apache.calcite.adapter.enumerable.CallImplementor;
+import org.apache.calcite.adapter.enumerable.NotNullImplementor;
+import org.apache.calcite.adapter.enumerable.NullPolicy;
+import org.apache.calcite.adapter.enumerable.RexImpTable;
+import org.apache.calcite.adapter.enumerable.RexToLixTranslator;
+import org.apache.calcite.linq4j.Enumerable;
+import org.apache.calcite.linq4j.Linq4j;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.linq4j.tree.Types;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.schema.FunctionParameter;
+import org.apache.calcite.schema.ImplementableFunction;
+import org.apache.calcite.schema.TableFunction;
+import org.apache.calcite.schema.impl.ReflectiveFunctionBase;
+import org.apache.calcite.sql.validate.SqlMoniker;
+import org.apache.calcite.util.BuiltInMethod;
+
+import com.google.common.collect.Iterables;
 
 import java.lang.reflect.Method;
 import java.lang.reflect.Type;
@@ -22,22 +45,6 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.reltype.RelDataTypeFactory;
-import org.eigenbase.rex.RexCall;
-import org.eigenbase.sql.validate.SqlMoniker;
-
-import net.hydromatic.linq4j.*;
-import net.hydromatic.linq4j.expressions.Expression;
-import net.hydromatic.linq4j.expressions.Expressions;
-import net.hydromatic.linq4j.expressions.Types;
-
-import net.hydromatic.optiq.*;
-import net.hydromatic.optiq.impl.ReflectiveFunctionBase;
-import net.hydromatic.optiq.rules.java.*;
-
-import com.google.common.collect.Iterables;
-
 /**
  * Table function that returns completion hints for a given SQL statement.
  */
@@ -46,7 +53,7 @@ public class SqlAdvisorGetHintsFunction
   private static final Expression ADVISOR =
       Expressions.convert_(
           Expressions.call(DataContext.ROOT,
-              BuiltinMethod.DATA_CONTEXT_GET.method,
+              BuiltInMethod.DATA_CONTEXT_GET.method,
               Expressions.constant(DataContext.Variable.SQL_ADVISOR.camelName)),
           SqlAdvisor.class);
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/advise/SqlAdvisorHint.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/advise/SqlAdvisorHint.java b/core/src/main/java/org/apache/calcite/sql/advise/SqlAdvisorHint.java
index 00d40d0..95bb404 100644
--- a/core/src/main/java/org/apache/calcite/sql/advise/SqlAdvisorHint.java
+++ b/core/src/main/java/org/apache/calcite/sql/advise/SqlAdvisorHint.java
@@ -14,22 +14,22 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.advise;
+package org.apache.calcite.sql.advise;
 
-import java.util.List;
+import org.apache.calcite.sql.validate.SqlMoniker;
 
-import org.eigenbase.sql.validate.SqlMoniker;
+import java.util.List;
 
 /**
- * This class is used to return values for {@link SqlAdvisor#getCompletionHints
- * (String, int, String[])}.
+ * This class is used to return values for
+ * {@link SqlAdvisor#getCompletionHints (String, int, String[])}.
  */
 public class SqlAdvisorHint {
   /** Fully qualified object name as string. */
   public final String id;
   /** Fully qualified object name as array of names. */
   public final String[] names;
-  /** One of {@link org.eigenbase.sql.validate.SqlMonikerType}. */
+  /** One of {@link org.apache.calcite.sql.validate.SqlMonikerType}. */
   public final String type;
 
   public SqlAdvisorHint(String id, String[] names, String type) {
@@ -47,3 +47,5 @@ public class SqlAdvisorHint {
     type = id.getType().name();
   }
 }
+
+// End SqlAdvisorHint.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/advise/SqlAdvisorValidator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/advise/SqlAdvisorValidator.java b/core/src/main/java/org/apache/calcite/sql/advise/SqlAdvisorValidator.java
index 78de9b7..b8ddaf4 100644
--- a/core/src/main/java/org/apache/calcite/sql/advise/SqlAdvisorValidator.java
+++ b/core/src/main/java/org/apache/calcite/sql/advise/SqlAdvisorValidator.java
@@ -14,16 +14,28 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.advise;
-
-import java.util.*;
-
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.sql.validate.*;
-import org.eigenbase.util.*;
+package org.apache.calcite.sql.advise;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.runtime.CalciteException;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperatorTable;
+import org.apache.calcite.sql.SqlSelect;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.SqlTypeUtil;
+import org.apache.calcite.sql.validate.OverScope;
+import org.apache.calcite.sql.validate.SqlConformance;
+import org.apache.calcite.sql.validate.SqlValidatorCatalogReader;
+import org.apache.calcite.sql.validate.SqlValidatorImpl;
+import org.apache.calcite.sql.validate.SqlValidatorNamespace;
+import org.apache.calcite.sql.validate.SqlValidatorScope;
+import org.apache.calcite.util.Util;
+
+import java.util.HashSet;
+import java.util.Set;
 
 /**
  * <code>SqlAdvisorValidator</code> is used by {@link SqlAdvisor} to traverse
@@ -70,7 +82,7 @@ public class SqlAdvisorValidator extends SqlValidatorImpl {
     registerId(id, scope);
     try {
       super.validateIdentifier(id, scope);
-    } catch (EigenbaseException e) {
+    } catch (CalciteException e) {
       Util.swallow(e, TRACER);
     }
   }
@@ -111,7 +123,7 @@ public class SqlAdvisorValidator extends SqlValidatorImpl {
     // Util.permAssert that throws Error
     try {
       return super.deriveType(scope, operand);
-    } catch (EigenbaseException e) {
+    } catch (CalciteException e) {
       return unknownType;
     } catch (UnsupportedOperationException e) {
       return unknownType;
@@ -129,7 +141,7 @@ public class SqlAdvisorValidator extends SqlValidatorImpl {
       SqlValidatorScope scope) {
     try {
       super.validateFrom(node, targetRowType, scope);
-    } catch (EigenbaseException e) {
+    } catch (CalciteException e) {
       Util.swallow(e, TRACER);
     }
   }
@@ -140,7 +152,7 @@ public class SqlAdvisorValidator extends SqlValidatorImpl {
   protected void validateWhereClause(SqlSelect select) {
     try {
       super.validateWhereClause(select);
-    } catch (EigenbaseException e) {
+    } catch (CalciteException e) {
       Util.swallow(e, TRACER);
     }
   }
@@ -151,7 +163,7 @@ public class SqlAdvisorValidator extends SqlValidatorImpl {
   protected void validateHavingClause(SqlSelect select) {
     try {
       super.validateHavingClause(select);
-    } catch (EigenbaseException e) {
+    } catch (CalciteException e) {
       Util.swallow(e, TRACER);
     }
   }
@@ -167,7 +179,7 @@ public class SqlAdvisorValidator extends SqlValidatorImpl {
         opScope = overScope;
       }
       validateWindow(window, opScope, null);
-    } catch (EigenbaseException e) {
+    } catch (CalciteException e) {
       Util.swallow(e, TRACER);
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/advise/SqlSimpleParser.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/advise/SqlSimpleParser.java b/core/src/main/java/org/apache/calcite/sql/advise/SqlSimpleParser.java
index 1d186dc..0bf8ce2 100644
--- a/core/src/main/java/org/apache/calcite/sql/advise/SqlSimpleParser.java
+++ b/core/src/main/java/org/apache/calcite/sql/advise/SqlSimpleParser.java
@@ -14,9 +14,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.advise;
-
-import java.util.*;
+package org.apache.calcite.sql.advise;
+
+import java.util.ArrayList;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.Map;
 
 /**
  * A simple parser that takes an incomplete and turn it into a syntactically

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/advise/package-info.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/advise/package-info.java b/core/src/main/java/org/apache/calcite/sql/advise/package-info.java
index 0ca1782..a30a886 100644
--- a/core/src/main/java/org/apache/calcite/sql/advise/package-info.java
+++ b/core/src/main/java/org/apache/calcite/sql/advise/package-info.java
@@ -22,8 +22,8 @@
  * invalid. It is edited in a SQL editor user-interface.</p>
  *
  * <p>The advisor uses the validation and parser framework set up in
- * <code>org.eigenbase.sql.validate</code> package.
+ * <code>org.apache.calcite.sql.validate</code> package.
 */
-package org.eigenbase.sql.advise;
+package org.apache.calcite.sql.advise;
 
 // End package-info.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/fun/SqlAbstractTimeFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlAbstractTimeFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlAbstractTimeFunction.java
index 16e9e00..6469927 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlAbstractTimeFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlAbstractTimeFunction.java
@@ -14,14 +14,23 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.fun;
+package org.apache.calcite.sql.fun;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.sql.validate.*;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlOperatorBinding;
+import org.apache.calcite.sql.SqlSyntax;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.SqlOperandTypeChecker;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.type.SqlTypeUtil;
+import org.apache.calcite.sql.validate.SqlMonotonicity;
+import org.apache.calcite.sql.validate.SqlValidatorScope;
 
-import static org.eigenbase.util.Static.RESOURCE;
+import static org.apache.calcite.util.Static.RESOURCE;
 
 /**
  * Base class for time functions such as "LOCALTIME", "LOCALTIME(n)".

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/fun/SqlArrayQueryConstructor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlArrayQueryConstructor.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlArrayQueryConstructor.java
index 252d0fa..2ea6b0a 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlArrayQueryConstructor.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlArrayQueryConstructor.java
@@ -14,10 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.fun;
+package org.apache.calcite.sql.fun;
 
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.validate.*;
+import org.apache.calcite.sql.SqlKind;
 
 /**
  * Definition of the SQL:2003 standard ARRAY query constructor, <code>

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/fun/SqlArrayValueConstructor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlArrayValueConstructor.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlArrayValueConstructor.java
index 9ac4f7e..b51c18b 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlArrayValueConstructor.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlArrayValueConstructor.java
@@ -14,12 +14,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.fun;
+package org.apache.calcite.sql.fun;
 
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.sql.SqlKind;
-import org.eigenbase.sql.SqlOperatorBinding;
-import org.eigenbase.sql.type.SqlTypeUtil;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlOperatorBinding;
+import org.apache.calcite.sql.type.SqlTypeUtil;
 
 /**
  * Definition of the SQL:2003 standard ARRAY constructor, <code>MULTISET
@@ -30,8 +30,7 @@ public class SqlArrayValueConstructor extends SqlMultisetValueConstructor {
     super("ARRAY", SqlKind.ARRAY_VALUE_CONSTRUCTOR);
   }
 
-  @Override
-  public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
+  @Override public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
     RelDataType type =
         getComponentType(
             opBinding.getTypeFactory(),

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/fun/SqlAvgAggFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlAvgAggFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlAvgAggFunction.java
index 8a6bfe7..af78e8c 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlAvgAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlAvgAggFunction.java
@@ -14,16 +14,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.fun;
+package org.apache.calcite.sql.fun;
 
-import java.util.List;
-
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.type.*;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.sql.SqlAggFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.ReturnTypes;
 
 import com.google.common.collect.ImmutableList;
 
+import java.util.List;
+
 /**
  * <code>Avg</code> is an aggregator which returns the average of the values
  * which go into it. It has precisely one argument of numeric type
@@ -77,6 +81,7 @@ public class SqlAvgAggFunction extends SqlAggFunction {
     return subtype;
   }
 
+  /** Sub-type of aggregate function. */
   public enum Subtype {
     AVG,
     STDDEV_POP,

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/fun/SqlBetweenOperator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlBetweenOperator.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlBetweenOperator.java
index 1f981c7..4b982ad 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlBetweenOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlBetweenOperator.java
@@ -14,19 +14,36 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.fun;
-
-import java.util.*;
-
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.sql.util.*;
-import org.eigenbase.sql.validate.*;
-import org.eigenbase.util.*;
-
-import static org.eigenbase.util.Static.RESOURCE;
+package org.apache.calcite.sql.fun;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeComparability;
+import org.apache.calcite.sql.ExplicitOperatorBinding;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlCallBinding;
+import org.apache.calcite.sql.SqlInfixOperator;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlOperatorBinding;
+import org.apache.calcite.sql.SqlUtil;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.parser.SqlParserUtil;
+import org.apache.calcite.sql.type.ComparableOperandTypeChecker;
+import org.apache.calcite.sql.type.ReturnTypes;
+import org.apache.calcite.sql.type.SqlOperandTypeChecker;
+import org.apache.calcite.sql.type.SqlTypeUtil;
+import org.apache.calcite.sql.util.SqlBasicVisitor;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql.validate.SqlValidatorScope;
+import org.apache.calcite.util.ImmutableNullableList;
+import org.apache.calcite.util.Util;
+
+import java.util.List;
+
+import static org.apache.calcite.util.Static.RESOURCE;
 
 /**
  * Defines the BETWEEN operator.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/fun/SqlCase.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlCase.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlCase.java
index 35e3776..fab92b2 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlCase.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlCase.java
@@ -14,13 +14,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.fun;
+package org.apache.calcite.sql.fun;
 
-import java.util.List;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlNodeList;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.util.UnmodifiableArrayList;
 
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.util.UnmodifiableArrayList;
+import java.util.List;
 
 /**
  * A <code>SqlCase</code> is a node of a parse tree which represents a case

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/fun/SqlCaseOperator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlCaseOperator.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlCaseOperator.java
index 44c4e12..0b060a6 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlCaseOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlCaseOperator.java
@@ -14,20 +14,36 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.fun;
+package org.apache.calcite.sql.fun;
 
-import java.util.*;
-
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.sql.validate.*;
-import org.eigenbase.util.Pair;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlCallBinding;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlNodeList;
+import org.apache.calcite.sql.SqlOperandCountRange;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlOperatorBinding;
+import org.apache.calcite.sql.SqlSyntax;
+import org.apache.calcite.sql.SqlUtil;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.InferTypes;
+import org.apache.calcite.sql.type.SqlOperandCountRanges;
+import org.apache.calcite.sql.type.SqlTypeUtil;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql.validate.SqlValidatorScope;
+import org.apache.calcite.util.Pair;
 
 import com.google.common.collect.Iterables;
 
-import static org.eigenbase.util.Static.RESOURCE;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.calcite.util.Static.RESOURCE;
 
 /**
  * An operator describing a <code>CASE</code>, <code>NULLIF</code> or <code>
@@ -92,7 +108,7 @@ import static org.eigenbase.util.Static.RESOURCE;
  * but there's now no way to represent a call to COALESCE or NULLIF. All in all,
  * it would be better to have operators for COALESCE, NULLIF, and both simple
  * and switched forms of CASE, then translate to simple CASE when building the
- * {@link org.eigenbase.rex.RexNode} tree.</p>
+ * {@link org.apache.calcite.rex.RexNode} tree.</p>
  *
  * <p>The arguments are physically represented as follows:</p>
  *

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/fun/SqlCastFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlCastFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlCastFunction.java
index 47d1b89..7c7c7c4 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlCastFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlCastFunction.java
@@ -14,16 +14,35 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.fun;
+package org.apache.calcite.sql.fun;
 
-import java.util.*;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFamily;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlCallBinding;
+import org.apache.calcite.sql.SqlDynamicParam;
+import org.apache.calcite.sql.SqlFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlIntervalQualifier;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperandCountRange;
+import org.apache.calcite.sql.SqlOperatorBinding;
+import org.apache.calcite.sql.SqlSyntax;
+import org.apache.calcite.sql.SqlUtil;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.type.InferTypes;
+import org.apache.calcite.sql.type.SqlOperandCountRanges;
+import org.apache.calcite.sql.type.SqlTypeFamily;
+import org.apache.calcite.sql.type.SqlTypeUtil;
+import org.apache.calcite.sql.validate.SqlMonotonicity;
+import org.apache.calcite.sql.validate.SqlValidatorScope;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.sql.validate.*;
+import java.util.HashSet;
+import java.util.Set;
 
-import static org.eigenbase.util.Static.RESOURCE;
+import static org.apache.calcite.util.Static.RESOURCE;
 
 /**
  * SqlCastFunction. Note that the std functions are really singleton objects,
@@ -210,8 +229,7 @@ public class SqlCastFunction extends SqlFunction {
     writer.endFunCall(frame);
   }
 
-  @Override
-  public SqlMonotonicity getMonotonicity(
+  @Override public SqlMonotonicity getMonotonicity(
       SqlCall call,
       SqlValidatorScope scope) {
     RelDataTypeFamily castFrom =
@@ -227,6 +245,7 @@ public class SqlCastFunction extends SqlFunction {
 
   //~ Inner Classes ----------------------------------------------------------
 
+  /** Pair of source-target type families. */
   private class TypeFamilyCast {
     private final RelDataTypeFamily castFrom;
     private final RelDataTypeFamily castTo;
@@ -238,8 +257,7 @@ public class SqlCastFunction extends SqlFunction {
       this.castTo = castTo;
     }
 
-    @Override
-    public boolean equals(Object obj) {
+    @Override public boolean equals(Object obj) {
       // TODO Auto-generated method stub
       if (obj.getClass() != TypeFamilyCast.class) {
         return false;
@@ -249,8 +267,7 @@ public class SqlCastFunction extends SqlFunction {
           && this.castTo.equals(other.castTo);
     }
 
-    @Override
-    public int hashCode() {
+    @Override public int hashCode() {
       // TODO Auto-generated method stub
       return castFrom.hashCode() + castTo.hashCode();
     }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/fun/SqlCeilFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlCeilFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlCeilFunction.java
index 9f5773d..a984c94 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlCeilFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlCeilFunction.java
@@ -14,11 +14,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.fun;
+package org.apache.calcite.sql.fun;
 
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.sql.validate.*;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.ReturnTypes;
+import org.apache.calcite.sql.validate.SqlMonotonicity;
+import org.apache.calcite.sql.validate.SqlValidatorScope;
 
 /**
  * Support for the CEIL/CEILING builtin function.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/fun/SqlCoalesceFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlCoalesceFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlCoalesceFunction.java
index 92e2509..89f33af 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlCoalesceFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlCoalesceFunction.java
@@ -14,15 +14,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.fun;
+package org.apache.calcite.sql.fun;
 
-import java.util.List;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlNodeList;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.ReturnTypes;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.util.Util;
 
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.sql.validate.*;
-import org.eigenbase.util.Util;
+import java.util.List;
 
 /**
  * The <code>COALESCE</code> function.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/fun/SqlCollectionTableOperator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlCollectionTableOperator.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlCollectionTableOperator.java
index be42639..a8b8535 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlCollectionTableOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlCollectionTableOperator.java
@@ -14,18 +14,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.fun;
+package org.apache.calcite.sql.fun;
 
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.type.*;
+import org.apache.calcite.sql.SqlFunctionalOperator;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.ReturnTypes;
 
 /**
  * SqlCollectionTableOperator is the "table function derived table" operator. It
  * converts a table-valued function into a relation, e.g. "<code>SELECT * FROM
  * TABLE(ramp(5))</code>".
  *
- * <p>This operator has function syntax (with one argument), whereas {@link
- * SqlStdOperatorTable#EXPLICIT_TABLE} is a prefix operator.
+ * <p>This operator has function syntax (with one argument), whereas
+ * {@link SqlStdOperatorTable#EXPLICIT_TABLE} is a prefix operator.
  */
 public class SqlCollectionTableOperator extends SqlFunctionalOperator {
   //~ Static fields/initializers ---------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/fun/SqlColumnListConstructor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlColumnListConstructor.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlColumnListConstructor.java
index db939ed..56d908c 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlColumnListConstructor.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlColumnListConstructor.java
@@ -14,10 +14,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.fun;
+package org.apache.calcite.sql.fun;
 
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.type.*;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlSpecialOperator;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.ReturnTypes;
 
 /**
  * SqlColumnListConstructor defines the non-standard constructor used to pass a

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/fun/SqlConvertFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlConvertFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlConvertFunction.java
index 3327b27..efdacdd 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlConvertFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlConvertFunction.java
@@ -14,9 +14,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.fun;
+package org.apache.calcite.sql.fun;
 
-import org.eigenbase.sql.*;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlWriter;
 
 /**
  * Common base for the <code>CONVERT</code> and <code>TRANSLATE</code>

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/fun/SqlCountAggFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlCountAggFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlCountAggFunction.java
index 319559f..0d9e1a5 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlCountAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlCountAggFunction.java
@@ -14,17 +14,25 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.fun;
+package org.apache.calcite.sql.fun;
 
-import java.util.List;
-
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.sql.validate.*;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.sql.SqlAggFunction;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlSyntax;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.ReturnTypes;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql.validate.SqlValidatorScope;
 
 import com.google.common.collect.ImmutableList;
 
+import java.util.List;
+
 /**
  * Definition of the SQL <code>COUNT</code> aggregation function.
  *

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/fun/SqlCovarAggFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlCovarAggFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlCovarAggFunction.java
index f759d0c..be63c1a 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlCovarAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlCovarAggFunction.java
@@ -14,20 +14,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.fun;
+package org.apache.calcite.sql.fun;
 
-import java.util.List;
-
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.reltype.RelDataTypeFactory;
-import org.eigenbase.sql.SqlAggFunction;
-import org.eigenbase.sql.SqlFunctionCategory;
-import org.eigenbase.sql.SqlKind;
-import org.eigenbase.sql.type.OperandTypes;
-import org.eigenbase.sql.type.ReturnTypes;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.sql.SqlAggFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.ReturnTypes;
 
 import com.google.common.collect.ImmutableList;
 
+import java.util.List;
+
 /**
  * <code>Covar</code> is an aggregator which returns the Covariance of the
  * values which go into it. It has precisely two arguments of numeric type

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/fun/SqlCurrentDateFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlCurrentDateFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlCurrentDateFunction.java
index fd86dba..cdab873 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlCurrentDateFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlCurrentDateFunction.java
@@ -14,11 +14,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.fun;
-
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.sql.validate.*;
+package org.apache.calcite.sql.fun;
+
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlSyntax;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.ReturnTypes;
+import org.apache.calcite.sql.validate.SqlMonotonicity;
+import org.apache.calcite.sql.validate.SqlValidatorScope;
 
 /**
  * The <code>CURRENT_DATE</code> function.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/fun/SqlCursorConstructor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlCursorConstructor.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlCursorConstructor.java
index 5ea0fff..a1fb812 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlCursorConstructor.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlCursorConstructor.java
@@ -14,12 +14,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.fun;
+package org.apache.calcite.sql.fun;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.sql.validate.*;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlSelect;
+import org.apache.calcite.sql.SqlSpecialOperator;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.ReturnTypes;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql.validate.SqlValidatorScope;
 
 /**
  * SqlCursorConstructor defines the non-standard CURSOR(&lt;query&gt;)

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/fun/SqlDatetimeSubtractionOperator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlDatetimeSubtractionOperator.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlDatetimeSubtractionOperator.java
index e577f1e..9ff4ee1 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlDatetimeSubtractionOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlDatetimeSubtractionOperator.java
@@ -14,12 +14,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.fun;
+package org.apache.calcite.sql.fun;
 
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.sql.validate.SqlMonotonicity;
-import org.eigenbase.sql.validate.SqlValidatorScope;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlSpecialOperator;
+import org.apache.calcite.sql.SqlSyntax;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.type.InferTypes;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.ReturnTypes;
+import org.apache.calcite.sql.validate.SqlMonotonicity;
+import org.apache.calcite.sql.validate.SqlValidatorScope;
 
 /**
  * A special operator for the subtraction of two DATETIMEs. The format of

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/fun/SqlExtractFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlExtractFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlExtractFunction.java
index 5fc5ece..7e43262 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlExtractFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlExtractFunction.java
@@ -14,11 +14,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.fun;
+package org.apache.calcite.sql.fun;
 
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.util.*;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.ReturnTypes;
+import org.apache.calcite.util.Util;
 
 /**
  * The SQL <code>EXTRACT</code> operator. Extracts a specified field value from

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/fun/SqlFirstLastValueAggFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlFirstLastValueAggFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlFirstLastValueAggFunction.java
index f401c58..5d382c3 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlFirstLastValueAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlFirstLastValueAggFunction.java
@@ -14,16 +14,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.fun;
+package org.apache.calcite.sql.fun;
 
-import java.util.List;
-
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.type.*;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.sql.SqlAggFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.ReturnTypes;
+import org.apache.calcite.sql.type.SqlTypeName;
 
 import com.google.common.collect.ImmutableList;
 
+import java.util.List;
+
 /**
  * <code>FIRST_VALUE</code> and <code>LAST_VALUE</code> aggregate functions
  * return the first or the last value in a list of values that are input to the

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/fun/SqlFloorFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlFloorFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlFloorFunction.java
index 04a14a2..bec351c 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlFloorFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlFloorFunction.java
@@ -14,11 +14,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.fun;
+package org.apache.calcite.sql.fun;
 
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.sql.validate.*;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.ReturnTypes;
+import org.apache.calcite.sql.validate.SqlMonotonicity;
+import org.apache.calcite.sql.validate.SqlValidatorScope;
 
 /**
  * Definition of the "FLOOR" builtin SQL function.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/fun/SqlHistogramAggFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlHistogramAggFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlHistogramAggFunction.java
index a9a24b9..9f2e461 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlHistogramAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlHistogramAggFunction.java
@@ -14,16 +14,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.fun;
+package org.apache.calcite.sql.fun;
 
-import java.util.List;
-
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.type.*;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.sql.SqlAggFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.ReturnTypes;
 
 import com.google.common.collect.ImmutableList;
 
+import java.util.List;
+
 /**
  * <code>HistogramAgg</code> is base operator which supports the Histogram
  * MIN/MAX aggregatoin functions. which returns the sum of the values which go

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/fun/SqlInOperator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlInOperator.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlInOperator.java
index b120d78..547e0e7 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlInOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlInOperator.java
@@ -14,18 +14,33 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.fun;
-
-import java.util.*;
-
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.sql.validate.*;
+package org.apache.calcite.sql.fun;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.sql.ExplicitOperatorBinding;
+import org.apache.calcite.sql.SqlBinaryOperator;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlCallBinding;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlNodeList;
+import org.apache.calcite.sql.type.ComparableOperandTypeChecker;
+import org.apache.calcite.sql.type.InferTypes;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.ReturnTypes;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.type.SqlTypeUtil;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql.validate.SqlValidatorScope;
 
 import com.google.common.collect.ImmutableList;
 
-import static org.eigenbase.util.Static.RESOURCE;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.calcite.util.Static.RESOURCE;
 
 /**
  * Definition of the SQL <code>IN</code> operator, which tests for a value's

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/fun/SqlItemOperator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlItemOperator.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlItemOperator.java
index 639665a..fb4d127 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlItemOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlItemOperator.java
@@ -14,15 +14,26 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.fun;
+package org.apache.calcite.sql.fun;
 
-import java.util.List;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlCallBinding;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperandCountRange;
+import org.apache.calcite.sql.SqlOperatorBinding;
+import org.apache.calcite.sql.SqlSpecialOperator;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.parser.SqlParserUtil;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.SqlOperandCountRanges;
+import org.apache.calcite.sql.type.SqlSingleOperandTypeChecker;
+import org.apache.calcite.sql.type.SqlTypeFamily;
+import org.apache.calcite.sql.type.SqlTypeName;
 
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.reltype.RelDataTypeFactory;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.parser.SqlParserUtil;
-import org.eigenbase.sql.type.*;
+import java.util.List;
 
 /**
  * The item operator {@code [ ... ]}, used to access a given element of an
@@ -40,8 +51,7 @@ class SqlItemOperator extends SqlSpecialOperator {
     super("ITEM", SqlKind.OTHER_FUNCTION, 100, true, null, null, null);
   }
 
-  @Override
-  public int reduceExpr(int ordinal, List<Object> list) {
+  @Override public int reduceExpr(int ordinal, List<Object> list) {
     SqlNode left = (SqlNode) list.get(ordinal - 1);
     SqlNode right = (SqlNode) list.get(ordinal + 1);
     final SqlParserUtil.ToTreeListItem treeListItem =
@@ -59,8 +69,7 @@ class SqlItemOperator extends SqlSpecialOperator {
     return ordinal - 1;
   }
 
-  @Override
-  public void unparse(
+  @Override public void unparse(
       SqlWriter writer, SqlCall call, int leftPrec, int rightPrec) {
     call.operand(0).unparse(writer, leftPrec, 0);
     final SqlWriter.Frame frame = writer.startList("[", "]");
@@ -68,13 +77,11 @@ class SqlItemOperator extends SqlSpecialOperator {
     writer.endList(frame);
   }
 
-  @Override
-  public SqlOperandCountRange getOperandCountRange() {
+  @Override public SqlOperandCountRange getOperandCountRange() {
     return SqlOperandCountRanges.of(2);
   }
 
-  @Override
-  public boolean checkOperandTypes(
+  @Override public boolean checkOperandTypes(
       SqlCallBinding callBinding,
       boolean throwOnFailure) {
     final SqlNode left = callBinding.getCall().operand(0);
@@ -105,14 +112,12 @@ class SqlItemOperator extends SqlSpecialOperator {
     }
   }
 
-  @Override
-  public String getAllowedSignatures(String name) {
+  @Override public String getAllowedSignatures(String name) {
     return "<ARRAY>[<INTEGER>]\n"
         + "<MAP>[<VALUE>]";
   }
 
-  @Override
-  public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
+  @Override public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
     final RelDataTypeFactory typeFactory = opBinding.getTypeFactory();
     final RelDataType operandType = opBinding.getOperandType(0);
     switch (operandType.getSqlTypeName()) {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/fun/SqlLeadLagAggFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlLeadLagAggFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlLeadLagAggFunction.java
index 5f91738..a6d5fec 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlLeadLagAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlLeadLagAggFunction.java
@@ -14,17 +14,27 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.fun;
+package org.apache.calcite.sql.fun;
 
-import java.util.List;
-
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.reltype.RelDataTypeFactory;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.type.*;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.sql.SqlAggFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlOperatorBinding;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.ReturnTypes;
+import org.apache.calcite.sql.type.SameOperandTypeChecker;
+import org.apache.calcite.sql.type.SqlReturnTypeInference;
+import org.apache.calcite.sql.type.SqlSingleOperandTypeChecker;
+import org.apache.calcite.sql.type.SqlTypeFamily;
+import org.apache.calcite.sql.type.SqlTypeTransform;
+import org.apache.calcite.sql.type.SqlTypeTransforms;
 
 import com.google.common.collect.ImmutableList;
 
+import java.util.List;
+
 /**
  * <code>LEAD</code> and <code>LAG</code> aggregate functions
  * return the value of given expression evaluated at given offset.
@@ -35,12 +45,12 @@ public class SqlLeadLagAggFunction extends SqlAggFunction {
           OperandTypes.ANY,
           OperandTypes.family(SqlTypeFamily.ANY, SqlTypeFamily.NUMERIC),
           OperandTypes.and(
-              OperandTypes.family(
-                  SqlTypeFamily.ANY, SqlTypeFamily.NUMERIC, SqlTypeFamily.ANY)
+              OperandTypes.family(SqlTypeFamily.ANY, SqlTypeFamily.NUMERIC,
+                  SqlTypeFamily.ANY),
               // Arguments 1 and 3 must have same type
-              , new SameOperandTypeChecker(3) {
-                @Override
-                protected List<Integer> getOperandList(int operandCount) {
+              new SameOperandTypeChecker(3) {
+                @Override protected List<Integer>
+                getOperandList(int operandCount) {
                   return ImmutableList.of(0, 2);
                 }
               }));

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 5de868a..ecd388a 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
@@ -14,14 +14,26 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.fun;
-
-import java.util.*;
-
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.util.*;
+package org.apache.calcite.sql.fun;
+
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlCallBinding;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperandCountRange;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlSpecialOperator;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.parser.SqlParserUtil;
+import org.apache.calcite.sql.type.InferTypes;
+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;
+
+import java.util.List;
 
 /**
  * An operator describing the <code>LIKE</code> and <code>SIMILAR</code>
@@ -36,9 +48,9 @@ import org.eigenbase.util.*;
  * escape-value]</code></li>
  * </ul>
  *
- * <p><b>NOTE</b> If the <code>NOT</code> clause is present the {@link
- * org.eigenbase.sql.parser.SqlParser parser} will generate a eqvivalent to
- * <code>NOT (src LIKE pattern ...)</code>
+ * <p><b>NOTE</b> If the <code>NOT</code> clause is present the
+ * {@link org.apache.calcite.sql.parser.SqlParser parser} will generate a
+ * eqvivalent to <code>NOT (src LIKE pattern ...)</code>
  */
 public class SqlLikeOperator extends SqlSpecialOperator {
   //~ Instance fields --------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/fun/SqlLiteralChainOperator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlLiteralChainOperator.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlLiteralChainOperator.java
index c6cafb9..d51ef0c 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlLiteralChainOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlLiteralChainOperator.java
@@ -14,20 +14,36 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.fun;
-
-import java.util.*;
-
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.sql.validate.*;
-import org.eigenbase.util.*;
-
-import net.hydromatic.linq4j.Ord;
-
-import static org.eigenbase.util.Static.RESOURCE;
+package org.apache.calcite.sql.fun;
+
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlCallBinding;
+import org.apache.calcite.sql.SqlCharStringLiteral;
+import org.apache.calcite.sql.SqlCollation;
+import org.apache.calcite.sql.SqlInternalOperator;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperatorBinding;
+import org.apache.calcite.sql.SqlUtil;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.InferTypes;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.ReturnTypes;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.type.SqlTypeUtil;
+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.NlsString;
+import org.apache.calcite.util.Util;
+
+import java.util.List;
+
+import static org.apache.calcite.util.Static.RESOURCE;
 
 /**
  * Internal operator, by which the parser represents a continued string literal.
@@ -36,8 +52,9 @@ import static org.eigenbase.util.Static.RESOURCE;
  * collected as the operands of an {@link SqlCall} using this operator. After
  * validation, the fragments will be concatenated into a single literal.
  *
- * <p>For a chain of {@link org.eigenbase.sql.SqlCharStringLiteral} objects, a
- * {@link SqlCollation} object is attached only to the head of the chain.
+ * <p>For a chain of {@link org.apache.calcite.sql.SqlCharStringLiteral}
+ * objects, a {@link SqlCollation} object is attached only to the head of the
+ * chain.
  */
 public class SqlLiteralChainOperator extends SqlInternalOperator {
   //~ Constructors -----------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/fun/SqlMapQueryConstructor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlMapQueryConstructor.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlMapQueryConstructor.java
index e3db960..e15421d 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlMapQueryConstructor.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlMapQueryConstructor.java
@@ -14,9 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.fun;
+package org.apache.calcite.sql.fun;
 
-import org.eigenbase.sql.SqlKind;
+import org.apache.calcite.sql.SqlKind;
 
 /**
  * Definition of the MAP query constructor, <code>

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/fun/SqlMapValueConstructor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlMapValueConstructor.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlMapValueConstructor.java
index e86df48..2e86321 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlMapValueConstructor.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlMapValueConstructor.java
@@ -14,18 +14,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.fun;
+package org.apache.calcite.sql.fun;
 
-import java.util.List;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.sql.SqlCallBinding;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlOperatorBinding;
+import org.apache.calcite.sql.type.SqlTypeUtil;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
 
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.reltype.RelDataTypeFactory;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.type.SqlTypeUtil;
-import org.eigenbase.util.Pair;
-import org.eigenbase.util.Util;
+import java.util.List;
 
-import static org.eigenbase.util.Static.RESOURCE;
+import static org.apache.calcite.util.Static.RESOURCE;
 
 /**
  * Definition of the MAP constructor,
@@ -38,8 +40,7 @@ public class SqlMapValueConstructor extends SqlMultisetValueConstructor {
     super("MAP", SqlKind.MAP_VALUE_CONSTRUCTOR);
   }
 
-  @Override
-  public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
+  @Override public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
     Pair<RelDataType, RelDataType> type =
         getComponentTypes(
             opBinding.getTypeFactory(), opBinding.collectOperandTypes());

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 0c231d0..74b3c99 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
@@ -14,14 +14,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.fun;
+package org.apache.calcite.sql.fun;
 
-import java.util.List;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.sql.SqlAggFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.ReturnTypes;
+import org.apache.calcite.util.Util;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.util.*;
+import java.util.List;
 
 /**
  * Definition of the <code>MIN</code> and <code>MAX</code> aggregate functions,

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/fun/SqlMonotonicBinaryOperator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlMonotonicBinaryOperator.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlMonotonicBinaryOperator.java
index 93595e2..42599a1 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlMonotonicBinaryOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlMonotonicBinaryOperator.java
@@ -14,11 +14,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.fun;
-
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.sql.validate.*;
+package org.apache.calcite.sql.fun;
+
+import org.apache.calcite.sql.SqlBinaryOperator;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.type.SqlOperandTypeChecker;
+import org.apache.calcite.sql.type.SqlOperandTypeInference;
+import org.apache.calcite.sql.type.SqlReturnTypeInference;
+import org.apache.calcite.sql.validate.SqlMonotonicity;
+import org.apache.calcite.sql.validate.SqlValidatorScope;
 
 /**
  * Base class for binary operators such as addition, subtraction, and

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/fun/SqlMonotonicUnaryFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlMonotonicUnaryFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlMonotonicUnaryFunction.java
index eefe699..d347fbc 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlMonotonicUnaryFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlMonotonicUnaryFunction.java
@@ -14,11 +14,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.fun;
+package org.apache.calcite.sql.fun;
 
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.sql.validate.*;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.type.SqlOperandTypeChecker;
+import org.apache.calcite.sql.type.SqlOperandTypeInference;
+import org.apache.calcite.sql.type.SqlReturnTypeInference;
+import org.apache.calcite.sql.validate.SqlMonotonicity;
+import org.apache.calcite.sql.validate.SqlValidatorScope;
 
 /**
  * Base class for unary operators such as FLOOR/CEIL which are monotonic for

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/fun/SqlMultisetMemberOfOperator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlMultisetMemberOfOperator.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlMultisetMemberOfOperator.java
index 866dff4..71b5880 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlMultisetMemberOfOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlMultisetMemberOfOperator.java
@@ -14,13 +14,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.fun;
+package org.apache.calcite.sql.fun;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.type.*;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlBinaryOperator;
+import org.apache.calcite.sql.SqlCallBinding;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlOperandCountRange;
+import org.apache.calcite.sql.type.MultisetSqlType;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.ReturnTypes;
+import org.apache.calcite.sql.type.SqlOperandCountRanges;
 
-import static org.eigenbase.util.Static.RESOURCE;
+import static org.apache.calcite.util.Static.RESOURCE;
 
 /**
  * Multiset MEMBER OF. Checks to see if a element belongs to a multiset.<br>


[40/58] [abbrv] [partial] incubator-calcite git commit: [CALCITE-306] Standardize code style for "import package.*; "

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 4d99b40..0b95f82 100644
--- a/core/src/main/java/org/apache/calcite/plan/SubstitutionVisitor.java
+++ b/core/src/main/java/org/apache/calcite/plan/SubstitutionVisitor.java
@@ -14,30 +14,48 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.relopt;
-
-import java.util.*;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.rel.rules.RemoveTrivialProjectRule;
-import org.eigenbase.reltype.*;
-import org.eigenbase.rex.*;
-import org.eigenbase.sql.SqlAggFunction;
-import org.eigenbase.sql.SqlKind;
-import org.eigenbase.sql.fun.SqlStdOperatorTable;
-import org.eigenbase.sql.validate.SqlValidatorUtil;
-import org.eigenbase.trace.EigenbaseTrace;
-import org.eigenbase.util.*;
-import org.eigenbase.util.mapping.Mapping;
-import org.eigenbase.util.mapping.Mappings;
-
-import net.hydromatic.linq4j.Ord;
-
-import net.hydromatic.optiq.prepare.OptiqPrepareImpl;
-import net.hydromatic.optiq.runtime.Spaces;
-import net.hydromatic.optiq.util.BitSets;
+package org.apache.calcite.plan;
+
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.prepare.CalcitePrepareImpl;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollationImpl;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.SingleRel;
+import org.apache.calcite.rel.core.Aggregate;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.rel.core.Values;
+import org.apache.calcite.rel.logical.LogicalAggregate;
+import org.apache.calcite.rel.logical.LogicalFilter;
+import org.apache.calcite.rel.logical.LogicalProject;
+import org.apache.calcite.rel.logical.LogicalUnion;
+import org.apache.calcite.rel.rules.ProjectRemoveRule;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexShuttle;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.runtime.Spaces;
+import org.apache.calcite.sql.SqlAggFunction;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.validate.SqlValidatorUtil;
+import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.ControlFlowException;
+import org.apache.calcite.util.IntList;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
+import org.apache.calcite.util.mapping.Mapping;
+import org.apache.calcite.util.mapping.Mappings;
+import org.apache.calcite.util.trace.CalciteTrace;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Equivalence;
@@ -45,7 +63,23 @@ import com.google.common.base.Function;
 import com.google.common.base.Objects;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Predicate;
-import com.google.common.collect.*;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.LinkedHashMultimap;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Multimap;
+import com.google.common.collect.Sets;
+
+import java.util.AbstractList;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.logging.Level;
+import java.util.logging.Logger;
 
 /**
  * Substitutes part of a tree of relational expressions with another tree.
@@ -71,17 +105,17 @@ import com.google.common.collect.*;
  * At each level, returns the residue.</p>
  *
  * <p>The inputs must only include the core relational operators:
- * {@link TableAccessRel},
- * {@link FilterRel},
- * {@link ProjectRel},
- * {@link JoinRel},
- * {@link UnionRel},
- * {@link AggregateRel}.</p>
+ * {@link org.apache.calcite.rel.logical.LogicalTableScan},
+ * {@link org.apache.calcite.rel.logical.LogicalFilter},
+ * {@link org.apache.calcite.rel.logical.LogicalProject},
+ * {@link org.apache.calcite.rel.logical.LogicalJoin},
+ * {@link org.apache.calcite.rel.logical.LogicalUnion},
+ * {@link org.apache.calcite.rel.logical.LogicalAggregate}.</p>
  */
 public class SubstitutionVisitor {
-  private static final boolean DEBUG = OptiqPrepareImpl.DEBUG;
+  private static final boolean DEBUG = CalcitePrepareImpl.DEBUG;
 
-  private static final Logger LOGGER = EigenbaseTrace.getPlannerTracer();
+  private static final Logger LOGGER = CalciteTrace.getPlannerTracer();
 
   /** Equivalence that compares objects by their {@link Object#toString()}
    * method. */
@@ -170,26 +204,26 @@ public class SubstitutionVisitor {
   }
 
   private static MutableRel toMutable(RelNode rel) {
-    if (rel instanceof TableAccessRelBase) {
-      return MutableScan.of((TableAccessRelBase) rel);
+    if (rel instanceof TableScan) {
+      return MutableScan.of((TableScan) rel);
     }
-    if (rel instanceof ValuesRelBase) {
-      return MutableValues.of((ValuesRelBase) rel);
+    if (rel instanceof Values) {
+      return MutableValues.of((Values) rel);
     }
-    if (rel instanceof ProjectRelBase) {
-      final ProjectRelBase project = (ProjectRelBase) rel;
-      final MutableRel input = toMutable(project.getChild());
+    if (rel instanceof Project) {
+      final Project project = (Project) rel;
+      final MutableRel input = toMutable(project.getInput());
       return MutableProject.of(input, project.getProjects(),
           project.getRowType().getFieldNames());
     }
-    if (rel instanceof FilterRelBase) {
-      final FilterRelBase filter = (FilterRelBase) rel;
-      final MutableRel input = toMutable(filter.getChild());
+    if (rel instanceof Filter) {
+      final Filter filter = (Filter) rel;
+      final MutableRel input = toMutable(filter.getInput());
       return MutableFilter.of(input, filter.getCondition());
     }
-    if (rel instanceof AggregateRelBase) {
-      final AggregateRelBase aggregate = (AggregateRelBase) rel;
-      final MutableRel input = toMutable(aggregate.getChild());
+    if (rel instanceof Aggregate) {
+      final Aggregate aggregate = (Aggregate) rel;
+      final MutableRel input = toMutable(aggregate.getInput());
       return MutableAggregate.of(input, aggregate.getGroupSet(),
           aggregate.getAggCallList());
     }
@@ -462,19 +496,18 @@ public class SubstitutionVisitor {
     final MutableRel node0 = unifyResult.result;
     MutableRel node = node0; // replaceAncestors(node0);
     if (DEBUG) {
-      System.out.println(
-          "Convert: query:\n"
-              + query.deep()
-              + "\nunify.query:\n"
-              + unifyResult.call.query.deep()
-              + "\nunify.result:\n"
-              + unifyResult.result.deep()
-              + "\nunify.target:\n"
-              + unifyResult.call.target.deep()
-              + "\nnode0:\n"
-              + node0.deep()
-              + "\nnode:\n"
-              + node.deep());
+      System.out.println("Convert: query:\n"
+          + query.deep()
+          + "\nunify.query:\n"
+          + unifyResult.call.query.deep()
+          + "\nunify.result:\n"
+          + unifyResult.result.deep()
+          + "\nunify.target:\n"
+          + unifyResult.call.target.deep()
+          + "\nnode0:\n"
+          + node0.deep()
+          + "\nnode:\n"
+          + node.deep());
     }
     return fromMutable(node);
   }
@@ -555,25 +588,26 @@ public class SubstitutionVisitor {
       return ((MutableLeafRel) node).rel;
     case PROJECT:
       final MutableProject project = (MutableProject) node;
-      return new ProjectRel(node.cluster,
+      return new LogicalProject(node.cluster,
           node.cluster.traitSetOf(RelCollationImpl.EMPTY),
           fromMutable(project.input),
-          project.projects, project.rowType, ProjectRelBase.Flags.BOXED);
+          project.projects, project.rowType, Project.Flags.BOXED);
     case FILTER:
       final MutableFilter filter = (MutableFilter) node;
-      return new FilterRel(node.cluster, fromMutable(filter.input),
+      return new LogicalFilter(node.cluster, fromMutable(filter.input),
           filter.condition);
     case AGGREGATE:
       final MutableAggregate aggregate = (MutableAggregate) node;
-      return new AggregateRel(node.cluster, fromMutable(aggregate.input),
+      return new LogicalAggregate(node.cluster, fromMutable(aggregate.input),
           aggregate.groupSet, aggregate.aggCalls);
     case SORT:
       final MutableSort sort = (MutableSort) node;
-      return new SortRel(node.cluster, node.cluster.traitSetOf(sort.collation),
+      return new Sort(node.cluster, node.cluster.traitSetOf(sort.collation),
           fromMutable(sort.input), sort.collation, sort.offset, sort.fetch);
     case UNION:
       final MutableUnion union = (MutableUnion) node;
-      return new UnionRel(union.cluster, fromMutables(union.inputs), union.all);
+      return new LogicalUnion(union.cluster, fromMutables(union.inputs),
+          union.all);
     default:
       throw new AssertionError(node.deep());
     }
@@ -598,8 +632,7 @@ public class SubstitutionVisitor {
           final UnifyResult x = apply(rule, queryLeaf, target);
           if (x != null) {
             if (DEBUG) {
-              System.out.println(
-                  "Rule: " + rule
+              System.out.println("Rule: " + rule
                   + "\nQuery:\n"
                   + queryParent
                   + (x.call.query != queryParent
@@ -869,7 +902,7 @@ public class SubstitutionVisitor {
    *
    * <p>Matches scans to the same table, because these will be
    * {@link MutableScan}s with the same
-   * {@link org.eigenbase.rel.TableAccessRel} instance.</p>
+   * {@link org.apache.calcite.rel.logical.LogicalTableScan} instance.</p>
    */
   private static class TrivialRule extends AbstractUnifyRule {
     private static final TrivialRule INSTANCE = new TrivialRule();
@@ -886,7 +919,8 @@ public class SubstitutionVisitor {
     }
   }
 
-  /** Implementation of {@link UnifyRule} that matches {@link ProjectRel}. */
+  /** Implementation of {@link UnifyRule} that matches
+   * {@link org.apache.calcite.rel.logical.LogicalProject}. */
   private static class ProjectToProjectUnifyRule extends AbstractUnifyRule {
     public static final ProjectToProjectUnifyRule INSTANCE =
         new ProjectToProjectUnifyRule();
@@ -1077,8 +1111,10 @@ public class SubstitutionVisitor {
     }
   }
 
-  /** Implementation of {@link UnifyRule} that matches a {@link AggregateRel} to
-   * a {@link AggregateRel}, provided that they have the same child. */
+  /** Implementation of {@link UnifyRule} that matches a
+   * {@link org.apache.calcite.rel.logical.LogicalAggregate} to a
+   * {@link org.apache.calcite.rel.logical.LogicalAggregate}, provided
+   * that they have the same child. */
   private static class AggregateToAggregateUnifyRule extends AbstractUnifyRule {
     public static final AggregateToAggregateUnifyRule INSTANCE =
         new AggregateToAggregateUnifyRule();
@@ -1198,11 +1234,11 @@ public class SubstitutionVisitor {
     public UnifyResult apply(UnifyRuleCall call) {
       final MutableAggregate query = (MutableAggregate) call.query;
       final MutableAggregate target = (MutableAggregate) call.target;
-      if (!(query.getChild() instanceof MutableProject)) {
+      if (!(query.getInput() instanceof MutableProject)) {
         return null;
       }
-      final MutableProject project = (MutableProject) query.getChild();
-      if (project.getChild() != target.getChild()) {
+      final MutableProject project = (MutableProject) query.getInput();
+      if (project.getInput() != target.getInput()) {
         return null;
       }
       final Mappings.TargetMapping mapping = project.getMapping();
@@ -1210,18 +1246,18 @@ public class SubstitutionVisitor {
         return null;
       }
       final MutableAggregate aggregate2 =
-          permute(query, project.getChild(), mapping.inverse());
+          permute(query, project.getInput(), mapping.inverse());
       final MutableRel result = unifyAggregates(aggregate2, target);
       return result == null ? null : call.result(result);
     }
   }
 
-  public static SqlAggFunction getRollup(Aggregation aggregation) {
+  public static SqlAggFunction getRollup(SqlAggFunction aggregation) {
     if (aggregation == SqlStdOperatorTable.SUM
         || aggregation == SqlStdOperatorTable.MIN
         || aggregation == SqlStdOperatorTable.MAX
         || aggregation == SqlStdOperatorTable.SUM0) {
-      return (SqlAggFunction) aggregation;
+      return aggregation;
     } else if (aggregation == SqlStdOperatorTable.COUNT) {
       return SqlStdOperatorTable.SUM0;
     } else {
@@ -1426,18 +1462,19 @@ public class SubstitutionVisitor {
       visitor.visit(input);
     }
 
-    public MutableRel getChild() {
+    public MutableRel getInput() {
       return input;
     }
   }
 
-  /** Mutable equivalent of {@link TableAccessRel}. */
+  /** Mutable equivalent of
+   * {@link org.apache.calcite.rel.logical.LogicalTableScan}. */
   private static class MutableScan extends MutableLeafRel {
-    private MutableScan(TableAccessRelBase rel) {
+    private MutableScan(TableScan rel) {
       super(MutableRelType.SCAN, rel);
     }
 
-    static MutableScan of(TableAccessRelBase rel) {
+    static MutableScan of(TableScan rel) {
       return new MutableScan(rel);
     }
 
@@ -1457,13 +1494,13 @@ public class SubstitutionVisitor {
     }
   }
 
-  /** Mutable equivalent of {@link ValuesRelBase}. */
+  /** Mutable equivalent of {@link org.apache.calcite.rel.core.Values}. */
   private static class MutableValues extends MutableLeafRel {
-    private MutableValues(ValuesRelBase rel) {
+    private MutableValues(Values rel) {
       super(MutableRelType.VALUES, rel);
     }
 
-    static MutableValues of(ValuesRelBase rel) {
+    static MutableValues of(Values rel) {
       return new MutableValues(rel);
     }
 
@@ -1479,11 +1516,12 @@ public class SubstitutionVisitor {
 
     @Override public StringBuilder digest(StringBuilder buf) {
       return buf.append("Values(tuples: ")
-          .append(((ValuesRelBase) rel).getTuples()).append(")");
+          .append(((Values) rel).getTuples()).append(")");
     }
   }
 
-  /** Mutable equivalent of {@link ProjectRel}. */
+  /** Mutable equivalent of
+   * {@link org.apache.calcite.rel.logical.LogicalProject}. */
   private static class MutableProject extends MutableSingleRel {
     private final List<RexNode> projects;
 
@@ -1500,7 +1538,7 @@ public class SubstitutionVisitor {
     }
 
     /** Equivalent to
-     * {@link RelOptUtil#createProject(org.eigenbase.rel.RelNode, java.util.List, java.util.List)}
+     * {@link RelOptUtil#createProject(org.apache.calcite.rel.RelNode, java.util.List, java.util.List)}
      * for {@link MutableRel}. */
     static MutableRel of(MutableRel child, List<RexNode> exprList,
         List<String> fieldNameList) {
@@ -1540,12 +1578,13 @@ public class SubstitutionVisitor {
     }
 
     public Mappings.TargetMapping getMapping() {
-      return ProjectRelBase.getMapping(
+      return Project.getMapping(
           input.getRowType().getFieldCount(), projects);
     }
   }
 
-  /** Mutable equivalent of {@link FilterRel}. */
+  /** Mutable equivalent of
+   * {@link org.apache.calcite.rel.logical.LogicalFilter}. */
   private static class MutableFilter extends MutableSingleRel {
     private final RexNode condition;
 
@@ -1567,7 +1606,7 @@ public class SubstitutionVisitor {
     }
 
     @Override public int hashCode() {
-      return Util.hashV(input, condition.toString());
+      return Objects.hashCode(input, condition.toString());
     }
 
     @Override public StringBuilder digest(StringBuilder buf) {
@@ -1579,7 +1618,8 @@ public class SubstitutionVisitor {
     }
   }
 
-  /** Mutable equivalent of {@link AggregateRel}. */
+  /** Mutable equivalent of
+   * {@link org.apache.calcite.rel.logical.LogicalAggregate}. */
   private static class MutableAggregate extends MutableSingleRel {
     private final BitSet groupSet;
     private final List<AggregateCall> aggCalls;
@@ -1594,7 +1634,7 @@ public class SubstitutionVisitor {
     static MutableAggregate of(MutableRel input, BitSet groupSet,
         List<AggregateCall> aggCalls) {
       RelDataType rowType =
-          AggregateRelBase.deriveRowType(input.cluster.getTypeFactory(),
+          Aggregate.deriveRowType(input.cluster.getTypeFactory(),
               input.getRowType(), groupSet, aggCalls);
       return new MutableAggregate(input, rowType, groupSet, aggCalls);
     }
@@ -1608,7 +1648,7 @@ public class SubstitutionVisitor {
     }
 
     @Override public int hashCode() {
-      return Util.hashV(input, groupSet, aggCalls);
+      return Objects.hashCode(input, groupSet, aggCalls);
     }
 
     @Override public StringBuilder digest(StringBuilder buf) {
@@ -1625,7 +1665,7 @@ public class SubstitutionVisitor {
     }
   }
 
-  /** Mutable equivalent of {@link SortRel}. */
+  /** Mutable equivalent of {@link org.apache.calcite.rel.core.Sort}. */
   private static class MutableSort extends MutableSingleRel {
     private final RelCollation collation;
     private final RexNode offset;
@@ -1654,7 +1694,7 @@ public class SubstitutionVisitor {
     }
 
     @Override public int hashCode() {
-      return Util.hashV(input, collation, offset, fetch);
+      return Objects.hashCode(input, collation, offset, fetch);
     }
 
     @Override public StringBuilder digest(StringBuilder buf) {
@@ -1694,7 +1734,8 @@ public class SubstitutionVisitor {
     }
   }
 
-  /** Mutable equivalent of {@link UnionRel}. */
+  /** Mutable equivalent of
+   * {@link org.apache.calcite.rel.logical.LogicalUnion}. */
   private static class MutableUnion extends MutableSetOp {
     public boolean all;
 
@@ -1717,7 +1758,7 @@ public class SubstitutionVisitor {
     }
 
     @Override public int hashCode() {
-      return Util.hashV(type, inputs);
+      return Objects.hashCode(type, inputs);
     }
 
     @Override public StringBuilder digest(StringBuilder buf) {
@@ -1799,19 +1840,21 @@ public class SubstitutionVisitor {
       }
     }
 
-    /** Based on {@link RemoveTrivialProjectRule#strip}. */
+    /** Based on
+     * {@link org.apache.calcite.rel.rules.ProjectRemoveRule#strip}. */
     public static MutableRel strip(MutableProject project) {
-      return isTrivial(project) ? project.getChild() : project;
+      return isTrivial(project) ? project.getInput() : project;
     }
 
-    /** Based on {@link RemoveTrivialProjectRule#isTrivial(ProjectRelBase)}. */
+    /** Based on
+     * {@link org.apache.calcite.rel.rules.ProjectRemoveRule#isTrivial(org.apache.calcite.rel.core.Project)}. */
     public static boolean isTrivial(MutableProject project) {
-      MutableRel child = project.getChild();
+      MutableRel child = project.getInput();
       final RelDataType childRowType = child.getRowType();
       if (!childRowType.isStruct()) {
         return false;
       }
-      if (!RemoveTrivialProjectRule.isIdentity(
+      if (!ProjectRemoveRule.isIdentity(
           project.getProjects(),
           project.getRowType(),
           childRowType)) {
@@ -1820,7 +1863,8 @@ public class SubstitutionVisitor {
       return true;
     }
 
-    /** Equivalent to {@link RelOptUtil#createProject(org.eigenbase.rel.RelNode, java.util.List)}
+    /** Equivalent to
+     * {@link RelOptUtil#createProject(org.apache.calcite.rel.RelNode, java.util.List)}
      * for {@link MutableRel}. */
     public static MutableRel createProject(final MutableRel child,
         final List<Integer> posList) {
@@ -1844,7 +1888,7 @@ public class SubstitutionVisitor {
           });
     }
 
-    /** Equivalence to {@link org.eigenbase.relopt.RelOptUtil#createCastRel}
+    /** Equivalence to {@link org.apache.calcite.plan.RelOptUtil#createCastRel}
      * for {@link MutableRel}. */
     public static MutableRel createCastRel(MutableRel rel,
         RelDataType castRowType, boolean rename) {
@@ -1999,13 +2043,14 @@ public class SubstitutionVisitor {
   }
 
   /**
-   * Rule that converts a {@link FilterRel} on top of a {@link ProjectRel} into
-   * a trivial filter (on a boolean column).
+   * Rule that converts a {@link org.apache.calcite.rel.logical.LogicalFilter}
+   * on top of a {@link org.apache.calcite.rel.logical.LogicalProject} into a
+   * trivial filter (on a boolean column).
    */
   public static class FilterOnProjectRule extends RelOptRule {
-    private static final Predicate<FilterRel> PREDICATE =
-        new Predicate<FilterRel>() {
-          public boolean apply(FilterRel input) {
+    private static final Predicate<LogicalFilter> PREDICATE =
+        new Predicate<LogicalFilter>() {
+          public boolean apply(LogicalFilter input) {
             return input.getCondition() instanceof RexInputRef;
           }
         };
@@ -2015,13 +2060,13 @@ public class SubstitutionVisitor {
 
     private FilterOnProjectRule() {
       super(
-          operand(FilterRel.class, null, PREDICATE,
-              some(operand(ProjectRel.class, any()))));
+          operand(LogicalFilter.class, null, PREDICATE,
+              some(operand(LogicalProject.class, any()))));
     }
 
     public void onMatch(RelOptRuleCall call) {
-      final FilterRel filter = call.rel(0);
-      final ProjectRel project = call.rel(1);
+      final LogicalFilter filter = call.rel(0);
+      final LogicalProject project = call.rel(1);
 
       final List<RexNode> newProjects =
           new ArrayList<RexNode>(project.getProjects());
@@ -2035,7 +2080,7 @@ public class SubstitutionVisitor {
               .build();
       final RelNode newProject =
           project.copy(project.getTraitSet(),
-              project.getChild(),
+              project.getInput(),
               newProjects,
               newRowType);
 
@@ -2043,7 +2088,7 @@ public class SubstitutionVisitor {
           cluster.getRexBuilder().makeInputRef(newProject,
               newProjects.size() - 1);
 
-      call.transformTo(new FilterRel(cluster, newProject, newCondition));
+      call.transformTo(new LogicalFilter(cluster, newProject, newCondition));
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/plan/TableAccessMap.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/TableAccessMap.java b/core/src/main/java/org/apache/calcite/plan/TableAccessMap.java
index 77a0e52..d6ef1fd 100644
--- a/core/src/main/java/org/apache/calcite/plan/TableAccessMap.java
+++ b/core/src/main/java/org/apache/calcite/plan/TableAccessMap.java
@@ -14,11 +14,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.relopt;
+package org.apache.calcite.plan;
 
-import java.util.*;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelVisitor;
+import org.apache.calcite.rel.core.TableModify;
 
-import org.eigenbase.rel.*;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
 
 // TODO jvs 9-Mar-2006:  move this class to another package; it
 // doesn't really belong here.  Also, use a proper class for table
@@ -31,6 +37,7 @@ import org.eigenbase.rel.*;
 public class TableAccessMap {
   //~ Enums ------------------------------------------------------------------
 
+  /** Access mode. */
   public static enum Mode {
     /**
      * Table is not accessed at all.
@@ -161,8 +168,8 @@ public class TableAccessMap {
 
   //~ Inner Classes ----------------------------------------------------------
 
+  /** Visitor that finds all tables in a tree. */
   private class TableRelVisitor extends RelVisitor {
-    // implement RelVisitor
     public void visit(
         RelNode p,
         int ordinal,
@@ -176,10 +183,10 @@ public class TableAccessMap {
 
       // FIXME jvs 1-Feb-2006:  Don't rely on object type here;
       // eventually someone is going to write a rule which transforms
-      // to something which doesn't inherit TableModificationRelBase,
+      // to something which doesn't inherit TableModify,
       // and this will break.  Need to make this explicit in
       // the RelNode interface.
-      if (p instanceof TableModificationRelBase) {
+      if (p instanceof TableModify) {
         newAccess = Mode.WRITE_ACCESS;
       } else {
         newAccess = Mode.READ_ACCESS;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/plan/VisitorRelVisitor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/VisitorRelVisitor.java b/core/src/main/java/org/apache/calcite/plan/VisitorRelVisitor.java
index 26292b5..d4cd0f5 100644
--- a/core/src/main/java/org/apache/calcite/plan/VisitorRelVisitor.java
+++ b/core/src/main/java/org/apache/calcite/plan/VisitorRelVisitor.java
@@ -14,16 +14,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.relopt;
+package org.apache.calcite.plan;
 
-import java.util.List;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelVisitor;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexVisitor;
 
-import org.eigenbase.rel.*;
-import org.eigenbase.rex.*;
+import java.util.List;
 
 /**
- * Walks over a tree of {@link RelNode relational expressions}, walking a {@link
- * RexShuttle} over every expression in that tree.
+ * Walks over a tree of {@link RelNode relational expressions}, walking a
+ * {@link RexVisitor} over every expression in that tree.
  */
 public class VisitorRelVisitor extends RelVisitor {
   //~ Instance fields --------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/plan/hep/HepInstruction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/hep/HepInstruction.java b/core/src/main/java/org/apache/calcite/plan/hep/HepInstruction.java
index b6c6bd3..86272e8 100644
--- a/core/src/main/java/org/apache/calcite/plan/hep/HepInstruction.java
+++ b/core/src/main/java/org/apache/calcite/plan/hep/HepInstruction.java
@@ -14,11 +14,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.relopt.hep;
+package org.apache.calcite.plan.hep;
 
-import java.util.*;
+import org.apache.calcite.plan.RelOptRule;
 
-import org.eigenbase.relopt.*;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Set;
 
 /**
  * HepInstruction represents one instruction in a HepProgram. The actual
@@ -36,6 +38,7 @@ abstract class HepInstruction {
 
   //~ Inner Classes ----------------------------------------------------------
 
+  /** Instruction that executes all rules of a given class. */
   static class RuleClass<R extends RelOptRule> extends HepInstruction {
     Class<R> ruleClass;
 
@@ -58,6 +61,7 @@ abstract class HepInstruction {
     }
   }
 
+  /** Instruction that executes all rules in a given collection. */
   static class RuleCollection extends HepInstruction {
     /**
      * Collection of rules to apply.
@@ -69,6 +73,7 @@ abstract class HepInstruction {
     }
   }
 
+  /** Instruction that executes converter rules. */
   static class ConverterRules extends HepInstruction {
     boolean guaranteed;
 
@@ -83,6 +88,7 @@ abstract class HepInstruction {
     }
   }
 
+  /** Instruction that finds common relational sub-expressions. */
   static class CommonRelSubExprRules extends HepInstruction {
     Set<RelOptRule> ruleSet;
 
@@ -91,6 +97,7 @@ abstract class HepInstruction {
     }
   }
 
+  /** Instruction that executes a given rule. */
   static class RuleInstance extends HepInstruction {
     /**
      * Description to look for, or null if rule specified explicitly.
@@ -119,6 +126,7 @@ abstract class HepInstruction {
     }
   }
 
+  /** Instruction that sets match order. */
   static class MatchOrder extends HepInstruction {
     HepMatchOrder order;
 
@@ -127,6 +135,7 @@ abstract class HepInstruction {
     }
   }
 
+  /** Instruction that sets match limit. */
   static class MatchLimit extends HepInstruction {
     int limit;
 
@@ -135,6 +144,7 @@ abstract class HepInstruction {
     }
   }
 
+  /** Instruction that executes a sub-program. */
   static class Subprogram extends HepInstruction {
     HepProgram subprogram;
 
@@ -147,6 +157,7 @@ abstract class HepInstruction {
     }
   }
 
+  /** Instruction that begins a group. */
   static class BeginGroup extends HepInstruction {
     EndGroup endGroup;
 
@@ -158,6 +169,7 @@ abstract class HepInstruction {
     }
   }
 
+  /** Instruction that ends a group. */
   static class EndGroup extends HepInstruction {
     /**
      * Actual rule set instantiated during planning by collecting grouped

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/plan/hep/HepMatchOrder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/hep/HepMatchOrder.java b/core/src/main/java/org/apache/calcite/plan/hep/HepMatchOrder.java
index 49dba1a..2b5f1ae 100644
--- a/core/src/main/java/org/apache/calcite/plan/hep/HepMatchOrder.java
+++ b/core/src/main/java/org/apache/calcite/plan/hep/HepMatchOrder.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.relopt.hep;
+package org.apache.calcite.plan.hep;
 
 /**
  * HepMatchOrder specifies the order of graph traversal when looking for rule

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 e06f8c1..30e28b0 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
@@ -14,24 +14,52 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.relopt.hep;
-
-import java.util.*;
-import java.util.logging.*;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.rel.convert.*;
-import org.eigenbase.rel.metadata.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.util.*;
-
-import net.hydromatic.linq4j.function.Function2;
-import net.hydromatic.linq4j.function.Functions;
-
-import net.hydromatic.optiq.util.graph.*;
+package org.apache.calcite.plan.hep;
+
+import org.apache.calcite.linq4j.function.Function2;
+import org.apache.calcite.linq4j.function.Functions;
+import org.apache.calcite.plan.AbstractRelOptPlanner;
+import org.apache.calcite.plan.CommonRelSubExprRule;
+import org.apache.calcite.plan.Context;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptCostFactory;
+import org.apache.calcite.plan.RelOptCostImpl;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleOperand;
+import org.apache.calcite.plan.RelTrait;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.Converter;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.convert.TraitMatchingRule;
+import org.apache.calcite.rel.metadata.RelMetadataProvider;
+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.BreadthFirstIterator;
+import org.apache.calcite.util.graph.CycleDetector;
+import org.apache.calcite.util.graph.DefaultDirectedGraph;
+import org.apache.calcite.util.graph.DefaultEdge;
+import org.apache.calcite.util.graph.DepthFirstIterator;
+import org.apache.calcite.util.graph.DirectedGraph;
+import org.apache.calcite.util.graph.Graphs;
+import org.apache.calcite.util.graph.TopologicalOrderIterator;
 
 import com.google.common.collect.ImmutableList;
 
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.logging.Level;
+
 /**
  * HepPlanner is a heuristic implementation of the {@link RelOptPlanner}
  * interface.
@@ -218,8 +246,7 @@ public class HepPlanner extends AbstractRelOptPlanner {
       instruction.rule =
           getRuleByDescription(instruction.ruleDescription);
       if (LOGGER.isLoggable(Level.FINEST)) {
-        LOGGER.finest(
-            "Looking up rule with description "
+        LOGGER.finest("Looking up rule with description "
             + instruction.ruleDescription
             + ", found " + instruction.rule);
       }
@@ -504,7 +531,7 @@ public class HepPlanner extends AbstractRelOptPlanner {
     List<HepRelVertex> parents = Graphs.predecessorListOf(graph, vertex);
     for (HepRelVertex parent : parents) {
       RelNode parentRel = parent.getCurrentRel();
-      if (parentRel instanceof ConverterRel) {
+      if (parentRel instanceof Converter) {
         // We don't support converter chains.
         continue;
       }
@@ -653,7 +680,7 @@ public class HepPlanner extends AbstractRelOptPlanner {
     for (HepRelVertex parent : allParents) {
       if (parentTrait != null) {
         RelNode parentRel = parent.getCurrentRel();
-        if (parentRel instanceof ConverterRel) {
+        if (parentRel instanceof Converter) {
           // We don't support automatically chaining conversions.
           // Treating a converter as a candidate parent here
           // can cause the "iParentMatch" check below to
@@ -673,7 +700,7 @@ public class HepPlanner extends AbstractRelOptPlanner {
 
     // There's a chance that newVertex is the same as one
     // of the parents due to common subexpression recognition
-    // (e.g. the ProjectRel added by SwapJoinRule).  In that
+    // (e.g. the LogicalProject added by JoinCommuteRule).  In that
     // case, treat the transformation as a nop to avoid
     // creating a loop.
     int iParentMatch = parents.indexOf(newVertex);
@@ -911,7 +938,7 @@ public class HepPlanner extends AbstractRelOptPlanner {
 
     throw Util.newInternal(
         "Query graph cycle detected in HepPlanner:  "
-        + cyclicVertices);
+            + cyclicVertices);
   }
 
   private void dumpGraph() {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/plan/hep/HepProgram.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/hep/HepProgram.java b/core/src/main/java/org/apache/calcite/plan/hep/HepProgram.java
index f112ab4..c7f44df 100644
--- a/core/src/main/java/org/apache/calcite/plan/hep/HepProgram.java
+++ b/core/src/main/java/org/apache/calcite/plan/hep/HepProgram.java
@@ -14,16 +14,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.relopt.hep;
-
-import java.util.*;
+package org.apache.calcite.plan.hep;
 
 import com.google.common.collect.ImmutableList;
 
+import java.util.List;
+
 /**
- * HepProgram specifies the order in which rules should be attempted by {@link
- * HepPlanner}. Use {@link HepProgramBuilder} to create a new instance of
- * HepProgram.
+ * HepProgram specifies the order in which rules should be attempted by
+ * {@link HepPlanner}. Use {@link HepProgramBuilder} to create a new
+ * instance of HepProgram.
  *
  * <p>Note that the structure of a program is immutable, but the planner uses it
  * as read/write during planning, so a program can only be in use by a single
@@ -51,7 +51,7 @@ public class HepProgram {
 
   /**
    * Creates a new empty HepProgram. The program has an initial match order of
-   * {@link org.eigenbase.relopt.hep.HepMatchOrder#ARBITRARY}, and an initial
+   * {@link org.apache.calcite.plan.hep.HepMatchOrder#ARBITRARY}, and an initial
    * match limit of {@link #MATCH_UNTIL_FIXPOINT}.
    */
   HepProgram(List<HepInstruction> instructions) {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/plan/hep/HepProgramBuilder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/hep/HepProgramBuilder.java b/core/src/main/java/org/apache/calcite/plan/hep/HepProgramBuilder.java
index 4fd7307..b698090 100644
--- a/core/src/main/java/org/apache/calcite/plan/hep/HepProgramBuilder.java
+++ b/core/src/main/java/org/apache/calcite/plan/hep/HepProgramBuilder.java
@@ -14,12 +14,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.relopt.hep;
+package org.apache.calcite.plan.hep;
 
-import java.util.*;
+import org.apache.calcite.plan.CommonRelSubExprRule;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelOptRule;
 
-import org.eigenbase.rel.convert.*;
-import org.eigenbase.relopt.*;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
 
 /**
  * HepProgramBuilder creates instances of {@link HepProgram}.
@@ -36,9 +39,9 @@ public class HepProgramBuilder {
 
   /**
    * Creates a new HepProgramBuilder with an initially empty program. The
-   * program under construction has an initial match order of {@link
-   * HepMatchOrder#ARBITRARY}, and an initial match limit of {@link
-   * HepProgram#MATCH_UNTIL_FIXPOINT}.
+   * program under construction has an initial match order of
+   * {@link HepMatchOrder#ARBITRARY}, and an initial match limit of
+   * {@link HepProgram#MATCH_UNTIL_FIXPOINT}.
    */
   public HepProgramBuilder() {
   }
@@ -56,9 +59,9 @@ public class HepProgramBuilder {
    * so if more control is needed, use addRuleInstance instead.
    *
    * <p>Note that when this method is used, it is also necessary to add the
-   * actual rule objects of interest to the planner via {@link
-   * RelOptPlanner#addRule}. If the planner does not have any rules of the
-   * given class, this instruction is a nop.
+   * actual rule objects of interest to the planner via
+   * {@link RelOptPlanner#addRule}. If the planner does not have any
+   * rules of the given class, this instruction is a nop.
    *
    * <p>TODO: support classification via rule annotations.
    *
@@ -165,8 +168,9 @@ public class HepProgramBuilder {
   }
 
   /**
-   * Adds an instruction to attempt to match instances of {@link
-   * ConverterRule}, but only where a conversion is actually required.
+   * Adds an instruction to attempt to match instances of
+   * {@link org.apache.calcite.rel.convert.ConverterRule},
+   * but only where a conversion is actually required.
    *
    * @param guaranteed if true, use only guaranteed converters; if false, use
    *                   only non-guaranteed converters

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/plan/hep/HepRelMetadataProvider.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/hep/HepRelMetadataProvider.java b/core/src/main/java/org/apache/calcite/plan/hep/HepRelMetadataProvider.java
index 5f41708..e944dbe 100644
--- a/core/src/main/java/org/apache/calcite/plan/hep/HepRelMetadataProvider.java
+++ b/core/src/main/java/org/apache/calcite/plan/hep/HepRelMetadataProvider.java
@@ -14,10 +14,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.relopt.hep;
+package org.apache.calcite.plan.hep;
 
-import org.eigenbase.rel.*;
-import org.eigenbase.rel.metadata.*;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.metadata.Metadata;
+import org.apache.calcite.rel.metadata.RelMetadataProvider;
 
 import com.google.common.base.Function;
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/plan/hep/HepRelVertex.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/hep/HepRelVertex.java b/core/src/main/java/org/apache/calcite/plan/hep/HepRelVertex.java
index 30130b5..5e8fab4 100644
--- a/core/src/main/java/org/apache/calcite/plan/hep/HepRelVertex.java
+++ b/core/src/main/java/org/apache/calcite/plan/hep/HepRelVertex.java
@@ -14,17 +14,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.relopt.hep;
+package org.apache.calcite.plan.hep;
+
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.AbstractRelNode;
+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.Util;
 
 import java.util.BitSet;
 import java.util.List;
 
-import org.eigenbase.rel.*;
-import org.eigenbase.rel.metadata.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.util.*;
-
 /**
  * HepRelVertex wraps a real {@link RelNode} as a vertex in a DAG representing
  * the entire query expression.
@@ -52,42 +56,35 @@ public class HepRelVertex extends AbstractRelNode {
     currentRel.explain(pw);
   }
 
-  @Override
-  public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+  @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
     assert traitSet.equals(this.traitSet);
     assert inputs.equals(this.getInputs());
     return this;
   }
 
-  @Override
-  public RelOptCost computeSelfCost(RelOptPlanner planner) {
+  @Override public RelOptCost computeSelfCost(RelOptPlanner planner) {
     // HepRelMetadataProvider is supposed to intercept this
     // and redirect to the real rels.
     throw Util.newInternal("should never get here");
   }
 
-  @Override
-  public double getRows() {
+  @Override public double getRows() {
     return RelMetadataQuery.getRowCount(currentRel);
   }
 
-  @Override
-  protected RelDataType deriveRowType() {
+  @Override protected RelDataType deriveRowType() {
     return currentRel.getRowType();
   }
 
-  @Override
-  public boolean isDistinct() {
+  @Override public boolean isDistinct() {
     return currentRel.isDistinct();
   }
 
-  @Override
-  public boolean isKey(BitSet columns) {
+  @Override public boolean isKey(BitSet columns) {
     return currentRel.isKey(columns);
   }
 
-  @Override
-  protected String computeDigest() {
+  @Override protected String computeDigest() {
     return "HepRelVertex(" + currentRel + ")";
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/plan/hep/HepRuleCall.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/hep/HepRuleCall.java b/core/src/main/java/org/apache/calcite/plan/hep/HepRuleCall.java
index 708955f..7aacbb7 100644
--- a/core/src/main/java/org/apache/calcite/plan/hep/HepRuleCall.java
+++ b/core/src/main/java/org/apache/calcite/plan/hep/HepRuleCall.java
@@ -14,12 +14,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.relopt.hep;
+package org.apache.calcite.plan.hep;
 
-import java.util.*;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptRuleOperand;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelNode;
 
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
 
 /**
  * HepRuleCall implements {@link RelOptRuleCall} for a {@link HepPlanner}. It

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/plan/hep/package-info.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/hep/package-info.java b/core/src/main/java/org/apache/calcite/plan/hep/package-info.java
index 9ae8dbb..424ad80 100644
--- a/core/src/main/java/org/apache/calcite/plan/hep/package-info.java
+++ b/core/src/main/java/org/apache/calcite/plan/hep/package-info.java
@@ -17,8 +17,8 @@
 
 /**
  * Provides a heuristic planner implementation for the interfaces in
- * {@link org.eigenbase.relopt}.
+ * {@link org.apache.calcite.plan}.
  */
-package org.eigenbase.relopt.hep;
+package org.apache.calcite.plan.hep;
 
 // End package-info.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/plan/package-info.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/package-info.java b/core/src/main/java/org/apache/calcite/plan/package-info.java
index d18be5d..e6fecd2 100644
--- a/core/src/main/java/org/apache/calcite/plan/package-info.java
+++ b/core/src/main/java/org/apache/calcite/plan/package-info.java
@@ -19,6 +19,6 @@
  * Defines interfaces for constructing rule-based optimizers of
  * relational expressions.
  */
-package org.eigenbase.relopt;
+package org.apache.calcite.plan;
 
 // End package-info.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/plan/volcano/AbstractConverter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/volcano/AbstractConverter.java b/core/src/main/java/org/apache/calcite/plan/volcano/AbstractConverter.java
index 66d9bcc..a4097bb 100644
--- a/core/src/main/java/org/apache/calcite/plan/volcano/AbstractConverter.java
+++ b/core/src/main/java/org/apache/calcite/plan/volcano/AbstractConverter.java
@@ -14,28 +14,36 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.relopt.volcano;
+package org.apache.calcite.plan.volcano;
 
-import java.util.List;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelTrait;
+import org.apache.calcite.plan.RelTraitDef;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelWriter;
+import org.apache.calcite.rel.convert.ConverterImpl;
 
-import org.eigenbase.rel.*;
-import org.eigenbase.rel.convert.*;
-import org.eigenbase.relopt.*;
+import java.util.List;
 
 /**
  * Converts a relational expression to any given output convention.
  *
- * <p>Unlike most {@link ConverterRel}s, an abstract converter is always
- * abstract. You would typically create an <code>AbstractConverter</code> when
- * it is necessary to transform a relational expression immediately; later,
- * rules will transform it into relational expressions which can be implemented.
- * </p>
+ * <p>Unlike most {@link org.apache.calcite.rel.convert.Converter}s, an abstract
+ * converter is always abstract. You would typically create an
+ * <code>AbstractConverter</code> when it is necessary to transform a relational
+ * expression immediately; later, rules will transform it into relational
+ * expressions which can be implemented.
  *
  * <p>If an abstract converter cannot be satisfied immediately (because the
  * source subset is abstract), the set is flagged, so this converter will be
  * expanded as soon as a non-abstract relexp is added to the set.</p>
  */
-public class AbstractConverter extends ConverterRelImpl {
+public class AbstractConverter extends ConverterImpl {
   //~ Constructors -----------------------------------------------------------
 
   public AbstractConverter(
@@ -102,7 +110,7 @@ public class AbstractConverter extends ConverterRelImpl {
     public void onMatch(RelOptRuleCall call) {
       final VolcanoPlanner planner = (VolcanoPlanner) call.getPlanner();
       AbstractConverter converter = call.rel(0);
-      final RelNode child = converter.getChild();
+      final RelNode child = converter.getInput();
       RelNode converted =
           planner.changeTraitsUsingConverters(
               child,

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/plan/volcano/ChainedPhaseRuleMappingInitializer.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/volcano/ChainedPhaseRuleMappingInitializer.java b/core/src/main/java/org/apache/calcite/plan/volcano/ChainedPhaseRuleMappingInitializer.java
index 82d2c1e..db65891 100644
--- a/core/src/main/java/org/apache/calcite/plan/volcano/ChainedPhaseRuleMappingInitializer.java
+++ b/core/src/main/java/org/apache/calcite/plan/volcano/ChainedPhaseRuleMappingInitializer.java
@@ -14,15 +14,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.relopt.volcano;
+package org.apache.calcite.plan.volcano;
 
-import java.util.*;
+import java.util.Map;
+import java.util.Set;
 
 /**
- * ChainedPhaseRuleMappingInitializer is an abstract implementation of {@link
- * VolcanoPlannerPhaseRuleMappingInitializer} that allows additional rules to be
- * layered ontop of those configured by a subordinate
- * VolcanoPlannerPhaseRuleMappingInitializer.
+ * ChainedPhaseRuleMappingInitializer is an abstract implementation of
+ * {@link VolcanoPlannerPhaseRuleMappingInitializer} that allows additional
+ * rules to be layered on top of those configured by a subordinate
+ * {@link VolcanoPlannerPhaseRuleMappingInitializer}.
  *
  * @see VolcanoPlannerPhaseRuleMappingInitializer
  */

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/plan/volcano/RelSet.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/volcano/RelSet.java b/core/src/main/java/org/apache/calcite/plan/volcano/RelSet.java
index 833d6f8..d389fce 100644
--- a/core/src/main/java/org/apache/calcite/plan/volcano/RelSet.java
+++ b/core/src/main/java/org/apache/calcite/plan/volcano/RelSet.java
@@ -14,17 +14,25 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.relopt.volcano;
+package org.apache.calcite.plan.volcano;
 
-import java.util.*;
-import java.util.logging.*;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.trace.*;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.ConventionTraitDef;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptListener;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.util.trace.CalciteTrace;
 
 import com.google.common.collect.ImmutableList;
 
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.logging.Logger;
+
 /**
  * A <code>RelSet</code> is an equivalence-set of expressions; that is, a set of
  * expressions which have identical semantics. We are generally interested in
@@ -36,7 +44,7 @@ import com.google.common.collect.ImmutableList;
 class RelSet {
   //~ Static fields/initializers ---------------------------------------------
 
-  private static final Logger LOGGER = EigenbaseTrace.getPlannerTracer();
+  private static final Logger LOGGER = CalciteTrace.getPlannerTracer();
 
   //~ Instance fields --------------------------------------------------------
 
@@ -196,8 +204,8 @@ class RelSet {
 
   /**
    * Adds an expression <code>rel</code> to this set, without creating a
-   * {@link org.eigenbase.relopt.volcano.RelSubset}. (Called only from
-   * {@link org.eigenbase.relopt.volcano.RelSubset#add}.
+   * {@link org.apache.calcite.plan.volcano.RelSubset}. (Called only from
+   * {@link org.apache.calcite.plan.volcano.RelSubset#add}.
    *
    * @param rel Relational expression
    */

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 7d29950..294a29c 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
@@ -14,21 +14,36 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.relopt.volcano;
-
-import java.io.*;
-import java.util.*;
-import java.util.logging.*;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.rel.metadata.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.trace.*;
-import org.eigenbase.util.*;
-
-import net.hydromatic.linq4j.Linq4j;
-import net.hydromatic.linq4j.function.Predicate1;
+package org.apache.calcite.plan.volcano;
+
+import org.apache.calcite.linq4j.Linq4j;
+import org.apache.calcite.linq4j.function.Predicate1;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptListener;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.RelTrait;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.AbstractRelNode;
+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.Util;
+import org.apache.calcite.util.trace.CalciteTrace;
+
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.logging.Level;
+import java.util.logging.Logger;
 
 /**
  * A <code>RelSubset</code> is set of expressions in a set which have the same
@@ -38,7 +53,7 @@ import net.hydromatic.linq4j.function.Predicate1;
 public class RelSubset extends AbstractRelNode {
   //~ Static fields/initializers ---------------------------------------------
 
-  private static final Logger LOGGER = EigenbaseTrace.getPlannerTracer();
+  private static final Logger LOGGER = CalciteTrace.getPlannerTracer();
 
   //~ Instance fields --------------------------------------------------------
 
@@ -170,8 +185,7 @@ public class RelSubset extends AbstractRelNode {
     return false;
   }
 
-  @Override
-  public boolean isKey(BitSet columns) {
+  @Override public boolean isKey(BitSet columns) {
     for (RelNode rel : set.rels) {
       if (rel.isKey(columns)) {
         return true;
@@ -332,8 +346,7 @@ public class RelSubset extends AbstractRelNode {
       final RelOptCost cost = planner.getCost(rel);
       if (cost.isLt(bestCost)) {
         if (LOGGER.isLoggable(Level.FINER)) {
-          LOGGER.finer(
-              "Subset cost improved: subset [" + this
+          LOGGER.finer("Subset cost improved: subset [" + this
               + "] cost was " + bestCost + " now " + cost);
         }
 
@@ -435,8 +448,7 @@ public class RelSubset extends AbstractRelNode {
           // out why we reached impasse.
           StringWriter sw = new StringWriter();
           final PrintWriter pw = new PrintWriter(sw);
-          pw.println(
-              "Node [" + subset.getDescription()
+          pw.println("Node [" + subset.getDescription()
               + "] could not be implemented; planner state:\n");
           planner.dump(pw);
           pw.flush();

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/plan/volcano/RuleQueue.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/volcano/RuleQueue.java b/core/src/main/java/org/apache/calcite/plan/volcano/RuleQueue.java
index 3198d64..9a5427d 100644
--- a/core/src/main/java/org/apache/calcite/plan/volcano/RuleQueue.java
+++ b/core/src/main/java/org/apache/calcite/plan/volcano/RuleQueue.java
@@ -14,21 +14,37 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.relopt.volcano;
+package org.apache.calcite.plan.volcano;
 
-import java.io.*;
-import java.util.*;
-import java.util.logging.*;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.trace.*;
-import org.eigenbase.util.*;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptRuleOperand;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.util.ChunkList;
+import org.apache.calcite.util.Stacks;
+import org.apache.calcite.util.Util;
+import org.apache.calcite.util.trace.CalciteTrace;
 
 import com.google.common.collect.HashMultimap;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Multimap;
 
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.EnumMap;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
 /**
  * Priority queue of relexps whose rules have not been called, and rule-matches
  * which have not yet been acted upon.
@@ -36,7 +52,7 @@ import com.google.common.collect.Multimap;
 class RuleQueue {
   //~ Static fields/initializers ---------------------------------------------
 
-  private static final Logger LOGGER = EigenbaseTrace.getPlannerTracer();
+  private static final Logger LOGGER = CalciteTrace.getPlannerTracer();
 
   private static final Set<String> ALL_RULES = ImmutableSet.of("<ALL RULES>");
 
@@ -180,8 +196,8 @@ class RuleQueue {
   }
 
   /**
-   * Equivalent to {@link #recompute(RelSubset, boolean) recompute(subset,
-   * false)}.
+   * Equivalent to
+   * {@link #recompute(RelSubset, boolean) recompute(subset, false)}.
    */
   public void recompute(RelSubset subset) {
     recompute(subset, false);
@@ -541,7 +557,7 @@ class RuleQueue {
    *
    * <p>is a valid match.</p>
    *
-   * @throws org.eigenbase.util.Util.FoundOne on match
+   * @throws org.apache.calcite.util.Util.FoundOne on match
    */
   private void checkDuplicateSubsets(List<RelSubset> subsets,
       RelOptRuleOperand operand, RelNode[] rels) {
@@ -578,8 +594,7 @@ class RuleQueue {
     }
     final double importance = parentImportance * alpha;
     if (LOGGER.isLoggable(Level.FINEST)) {
-      LOGGER.finest(
-          "Importance of [" + child + "] to its parent ["
+      LOGGER.finest("Importance of [" + child + "] to its parent ["
           + parent + "] is " + importance + " (parent importance="
           + parentImportance + ", child cost=" + childCost
           + ", parent cost=" + parentCost + ")");
@@ -666,8 +681,8 @@ class RuleQueue {
 
   /**
    * PhaseMatchList represents a set of {@link VolcanoRuleMatch rule-matches}
-   * for a particular {@link VolcanoPlannerPhase phase of the planner's
-   * execution}.
+   * for a particular
+   * {@link VolcanoPlannerPhase phase of the planner's execution}.
    */
   private static class PhaseMatchList {
     /**

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoCost.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoCost.java b/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoCost.java
index fcab480..f933640 100644
--- a/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoCost.java
+++ b/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoCost.java
@@ -14,10 +14,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.relopt.volcano;
+package org.apache.calcite.plan.volcano;
 
-import org.eigenbase.relopt.*;
-import org.eigenbase.util.Util;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptCostFactory;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.util.Util;
 
 /**
  * <code>VolcanoCost</code> represents the cost of a plan node.
@@ -116,8 +118,7 @@ class VolcanoCost implements RelOptCost {
     return rowCount;
   }
 
-  @Override
-  public int hashCode() {
+  @Override public int hashCode() {
     return Util.hashCode(rowCount) + Util.hashCode(cpu) + Util.hashCode(io);
   }
 
@@ -206,6 +207,8 @@ class VolcanoCost implements RelOptCost {
     return "{" + rowCount + " rows, " + cpu + " cpu, " + io + " io}";
   }
 
+  /** Implementation of {@link org.apache.calcite.plan.RelOptCostFactory}
+   * that creates {@link org.apache.calcite.plan.volcano.VolcanoCost}s. */
   private static class Factory implements RelOptCostFactory {
     public RelOptCost makeCost(double dRows, double dCpu, double dIo) {
       return new VolcanoCost(dRows, dCpu, dIo);

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 03afff5..87623ea 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
@@ -14,37 +14,96 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.relopt.volcano;
-
-import java.io.*;
-import java.util.*;
-import java.util.logging.*;
-import java.util.regex.*;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.rel.convert.*;
-import org.eigenbase.rel.metadata.*;
-import org.eigenbase.rel.rules.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.relopt.hep.*;
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.sql.SqlExplainLevel;
-import org.eigenbase.util.*;
-
-import net.hydromatic.linq4j.expressions.Expressions;
-
-import net.hydromatic.optiq.config.OptiqConnectionConfig;
-import net.hydromatic.optiq.prepare.OptiqPrepareImpl;
-import net.hydromatic.optiq.runtime.Hook;
-import net.hydromatic.optiq.runtime.Spaces;
-import net.hydromatic.optiq.util.graph.*;
+package org.apache.calcite.plan.volcano;
+
+import org.apache.calcite.config.CalciteConnectionConfig;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.plan.AbstractRelOptPlanner;
+import org.apache.calcite.plan.Context;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptCostFactory;
+import org.apache.calcite.plan.RelOptLattice;
+import org.apache.calcite.plan.RelOptListener;
+import org.apache.calcite.plan.RelOptMaterialization;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptRuleOperand;
+import org.apache.calcite.plan.RelOptSchema;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.RelTrait;
+import org.apache.calcite.plan.RelTraitDef;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.plan.SubstitutionVisitor;
+import org.apache.calcite.plan.hep.HepPlanner;
+import org.apache.calcite.plan.hep.HepProgram;
+import org.apache.calcite.plan.hep.HepProgramBuilder;
+import org.apache.calcite.prepare.CalcitePrepareImpl;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelVisitor;
+import org.apache.calcite.rel.convert.Converter;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.rel.metadata.RelMetadataProvider;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rel.rules.AggregateRemoveRule;
+import org.apache.calcite.rel.rules.CalcRemoveRule;
+import org.apache.calcite.rel.rules.FilterJoinRule;
+import org.apache.calcite.rel.rules.FilterProjectTransposeRule;
+import org.apache.calcite.rel.rules.JoinAssociateRule;
+import org.apache.calcite.rel.rules.JoinCommuteRule;
+import org.apache.calcite.rel.rules.ProjectMergeRule;
+import org.apache.calcite.rel.rules.ProjectRemoveRule;
+import org.apache.calcite.rel.rules.SemiJoinRule;
+import org.apache.calcite.rel.rules.SortRemoveRule;
+import org.apache.calcite.rel.rules.UnionToDistinctRule;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.runtime.Hook;
+import org.apache.calcite.runtime.Spaces;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.SaffronProperties;
+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.Graphs;
 
 import com.google.common.base.Function;
 import com.google.common.base.Supplier;
 import com.google.common.base.Suppliers;
-import com.google.common.collect.*;
-
-import static org.eigenbase.util.Stacks.*;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.LinkedListMultimap;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Multimap;
+import com.google.common.collect.Sets;
+
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.IdentityHashMap;
+import java.util.Iterator;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.logging.Level;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import static org.apache.calcite.util.Stacks.peek;
+import static org.apache.calcite.util.Stacks.pop;
+import static org.apache.calcite.util.Stacks.push;
 
 /**
  * VolcanoPlanner optimizes queries by transforming expressions selectively
@@ -204,7 +263,7 @@ public class VolcanoPlanner extends AbstractRelOptPlanner {
       new ArrayList<VolcanoRuleCall>();
 
   /** Zero cost, according to {@link #costFactory}. Not necessarily a
-   * {@link org.eigenbase.relopt.volcano.VolcanoCost}. */
+   * {@link org.apache.calcite.plan.volcano.VolcanoCost}. */
   private final RelOptCost zeroCost;
 
   //~ Constructors -----------------------------------------------------------
@@ -327,8 +386,8 @@ public class VolcanoPlanner extends AbstractRelOptPlanner {
     RelNode target = materialization.queryRel;
     HepProgram program =
         new HepProgramBuilder()
-            .addRuleInstance(PushFilterPastProjectRule.INSTANCE)
-            .addRuleInstance(MergeProjectRule.INSTANCE)
+            .addRuleInstance(FilterProjectTransposeRule.INSTANCE)
+            .addRuleInstance(ProjectMergeRule.INSTANCE)
             .build();
 
     final HepPlanner hepPlanner = new HepPlanner(program, getContext());
@@ -344,8 +403,8 @@ public class VolcanoPlanner extends AbstractRelOptPlanner {
 
   private void useApplicableMaterializations() {
     // Avoid using materializations while populating materializations!
-    final OptiqConnectionConfig config =
-        context.unwrap(OptiqConnectionConfig.class);
+    final CalciteConnectionConfig config =
+        context.unwrap(CalciteConnectionConfig.class);
     if (config == null || !config.materializationsEnabled()) {
       return;
     }
@@ -403,7 +462,7 @@ public class VolcanoPlanner extends AbstractRelOptPlanner {
       if (queryTableNames.contains(lattice.rootTable().getQualifiedName())) {
         RelNode rel2 = lattice.rewrite(leafJoinRoot.get());
         if (rel2 != null) {
-          if (OptiqPrepareImpl.DEBUG) {
+          if (CalcitePrepareImpl.DEBUG) {
             System.out.println("use lattice:\n" + RelOptUtil.toString(rel2));
           }
           latticeUses.add(Pair.of(lattice, rel2));
@@ -436,9 +495,8 @@ public class VolcanoPlanner extends AbstractRelOptPlanner {
   private static Set<RelOptTable> findTables(RelNode rel) {
     final Set<RelOptTable> usedTables = new LinkedHashSet<RelOptTable>();
     new RelVisitor() {
-      @Override
-      public void visit(RelNode node, int ordinal, RelNode parent) {
-        if (node instanceof TableAccessRelBase) {
+      @Override public void visit(RelNode node, int ordinal, RelNode parent) {
+        if (node instanceof TableScan) {
           usedTables.add(node.getTable());
         }
         super.visit(node, ordinal, parent);
@@ -466,23 +524,19 @@ public class VolcanoPlanner extends AbstractRelOptPlanner {
     return null;
   }
 
-  @Override
-  public boolean addRelTraitDef(RelTraitDef relTraitDef) {
+  @Override public boolean addRelTraitDef(RelTraitDef relTraitDef) {
     return !traitDefs.contains(relTraitDef) && traitDefs.add(relTraitDef);
   }
 
-  @Override
-  public void clearRelTraitDefs() {
+  @Override public void clearRelTraitDefs() {
     traitDefs.clear();
   }
 
-  @Override
-  public List<RelTraitDef> getRelTraitDefs() {
+  @Override public List<RelTraitDef> getRelTraitDefs() {
     return traitDefs;
   }
 
-  @Override
-  public RelTraitSet emptyTraitSet() {
+  @Override public RelTraitSet emptyTraitSet() {
     RelTraitSet traitSet = super.emptyTraitSet();
     for (RelTraitDef traitDef : traitDefs) {
       if (traitDef.multiple()) {
@@ -629,7 +683,7 @@ public class VolcanoPlanner extends AbstractRelOptPlanner {
 
   /**
    * Finds the most efficient expression to implement the query given via
-   * {@link org.eigenbase.relopt.RelOptPlanner#setRoot(org.eigenbase.rel.RelNode)}.
+   * {@link org.apache.calcite.plan.RelOptPlanner#setRoot(org.apache.calcite.rel.RelNode)}.
    *
    * <p>The algorithm executes repeatedly in a series of phases. In each phase
    * the exact rules that may be fired varies. The mapping of phases to rule
@@ -708,8 +762,7 @@ public class VolcanoPlanner extends AbstractRelOptPlanner {
         }
 
         if (LOGGER.isLoggable(Level.FINE)) {
-          LOGGER.fine(
-              "PLANNER = " + this
+          LOGGER.fine("PLANNER = " + this
               + "; TICK = " + cumulativeTicks + "/" + tick
               + "; PHASE = " + phase.toString()
               + "; COST = " + root.bestCost);
@@ -782,7 +835,7 @@ public class VolcanoPlanner extends AbstractRelOptPlanner {
   }
 
   /**
-   * Helper for {@link #provenance(org.eigenbase.rel.RelNode)}.
+   * Helper for {@link #provenance(org.apache.calcite.rel.RelNode)}.
    */
   private void provenanceRecurse(
       PrintWriter pw, RelNode node, int i, Set<RelNode> visited) {
@@ -858,8 +911,8 @@ public class VolcanoPlanner extends AbstractRelOptPlanner {
   }
 
   /**
-   * Finds RelSubsets in the plan that contain only rels of {@link
-   * Convention#NONE} and boosts their importance by 25%.
+   * Finds RelSubsets in the plan that contain only rels of
+   * {@link Convention#NONE} and boosts their importance by 25%.
    */
   private void injectImportanceBoost() {
     HashSet<RelSubset> requireBoost = new HashSet<RelSubset>();
@@ -961,19 +1014,19 @@ public class VolcanoPlanner extends AbstractRelOptPlanner {
   }
 
   public void registerAbstractRelationalRules() {
-    addRule(PushFilterPastJoinRule.FILTER_ON_JOIN);
-    addRule(PushFilterPastJoinRule.JOIN);
+    addRule(FilterJoinRule.FILTER_ON_JOIN);
+    addRule(FilterJoinRule.JOIN);
     addRule(AbstractConverter.ExpandConversionRule.INSTANCE);
-    addRule(SwapJoinRule.INSTANCE);
+    addRule(JoinCommuteRule.INSTANCE);
     addRule(SemiJoinRule.INSTANCE);
-    if (OptiqPrepareImpl.COMMUTE) {
-      addRule(CommutativeJoinRule.INSTANCE);
+    if (CalcitePrepareImpl.COMMUTE) {
+      addRule(JoinAssociateRule.INSTANCE);
     }
-    addRule(RemoveDistinctRule.INSTANCE);
+    addRule(AggregateRemoveRule.INSTANCE);
     addRule(UnionToDistinctRule.INSTANCE);
-    addRule(RemoveTrivialProjectRule.INSTANCE);
-    addRule(RemoveTrivialCalcRule.INSTANCE);
-    addRule(RemoveSortRule.INSTANCE);
+    addRule(ProjectRemoveRule.INSTANCE);
+    addRule(CalcRemoveRule.INSTANCE);
+    addRule(SortRemoveRule.INSTANCE);
 
     // todo: rule which makes Project({OrdinalRef}) disappear
   }
@@ -1227,8 +1280,7 @@ public class VolcanoPlanner extends AbstractRelOptPlanner {
           }
         });
     for (RelSet set : sets) {
-      pw.println(
-          "Set#" + set.id
+      pw.println("Set#" + set.id
           + ", type: " + set.subsets.get(0).getRowType());
       int j = -1;
       for (RelSubset subset : set.subsets) {
@@ -1492,8 +1544,8 @@ public class VolcanoPlanner extends AbstractRelOptPlanner {
    * equivalence set. If an identical expression is already registered, we
    * don't need to register this one and nor should we queue up rule matches.
    *
-   * @param rel relational expression to register. Must be either a {@link
-   *            RelSubset}, or an unregistered {@link RelNode}
+   * @param rel relational expression to register. Must be either a
+   *         {@link RelSubset}, or an unregistered {@link RelNode}
    * @param set set that rel belongs to, or <code>null</code>
    * @return the equivalence-set
    * @pre rel instanceof RelSubset || !isRegistered(rel)
@@ -1509,8 +1561,7 @@ public class VolcanoPlanner extends AbstractRelOptPlanner {
     }
 
     if (rel.getCluster().getPlanner() != this) {
-      throw Util.newInternal(
-          "Relational expression " + rel
+      throw Util.newInternal("Relational expression " + rel
           + " belongs to a different planner than is currently being"
           + " used.");
     }
@@ -1521,12 +1572,12 @@ public class VolcanoPlanner extends AbstractRelOptPlanner {
     final Convention convention =
         (Convention) traits.getTrait(0);
     if (!convention.getInterface().isInstance(rel)
-        && !(rel instanceof ConverterRel)) {
+        && !(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");
+              + " has calling-convention " + convention
+              + " but does not implement the required interface '"
+              + convention.getInterface() + "' of that convention");
     }
     if (traits.size() != traitDefs.size()) {
       throw Util.newInternal(
@@ -1580,8 +1631,8 @@ public class VolcanoPlanner extends AbstractRelOptPlanner {
     }
 
     // Converters are in the same set as their children.
-    if (rel instanceof ConverterRel) {
-      final RelNode input = ((ConverterRel) rel).getChild();
+    if (rel instanceof Converter) {
+      final RelNode input = ((Converter) rel).getInput();
       final RelSet childSet = getSet(input);
       if ((set != null)
           && (set != childSet)
@@ -1719,8 +1770,7 @@ public class VolcanoPlanner extends AbstractRelOptPlanner {
         && (set != null)
         && (set.equivalentSet == null)
         && (subset.set.equivalentSet == null)) {
-      LOGGER.finer(
-          "Register #" + subset.getId() + " " + subset
+      LOGGER.finer("Register #" + subset.getId() + " " + subset
           + ", and merge sets");
       merge(set, subset.set);
       registerCount++;
@@ -1808,7 +1858,7 @@ public class VolcanoPlanner extends AbstractRelOptPlanner {
 
   /**
    * Sets whether this planner is locked. A locked planner does not accept
-   * new rules. {@link #addRule(org.eigenbase.relopt.RelOptRule)} will do
+   * new rules. {@link #addRule(org.apache.calcite.plan.RelOptRule)} will do
    * nothing and return false.
    *
    * @param locked Whether planner is locked

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoPlannerPhase.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoPlannerPhase.java b/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoPlannerPhase.java
index 68dfd31..13b51a1 100644
--- a/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoPlannerPhase.java
+++ b/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoPlannerPhase.java
@@ -14,12 +14,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.relopt.volcano;
+package org.apache.calcite.plan.volcano;
 
 /**
- * VolcanoPlannerPhase represents the phases of operation that the {@link
- * VolcanoPlanner} passes through during optimization of a tree of {@link
- * org.eigenbase.rel.RelNode} objects.
+ * VolcanoPlannerPhase represents the phases of operation that the
+ * {@link VolcanoPlanner} passes through during optimization of a tree of
+ * {@link org.apache.calcite.rel.RelNode} objects.
  */
 public enum VolcanoPlannerPhase {
   PRE_PROCESS_MDR, PRE_PROCESS, OPTIMIZE, CLEANUP,

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoPlannerPhaseRuleMappingInitializer.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoPlannerPhaseRuleMappingInitializer.java b/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoPlannerPhaseRuleMappingInitializer.java
index 49d96ea..3f839e2 100644
--- a/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoPlannerPhaseRuleMappingInitializer.java
+++ b/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoPlannerPhaseRuleMappingInitializer.java
@@ -14,21 +14,22 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.relopt.volcano;
+package org.apache.calcite.plan.volcano;
 
-import java.util.*;
+import java.util.Map;
+import java.util.Set;
 
 /**
  * VolcanoPlannerPhaseRuleMappingInitializer describes an inteface for
  * initializing the mapping of {@link VolcanoPlannerPhase}s to sets of rule
  * descriptions.
  *
- * <p><b>Note:</b> Rule descriptions are obtained via {@link
- * org.eigenbase.relopt.RelOptRule#toString()}. By default they are the class's
- * simple name (e.g. class name sans package), unless the class is an inner
- * class, in which case the default is the inner class's simple name. Some rules
- * explicitly provide alternate descriptions by calling the
- * {@link org.eigenbase.relopt.RelOptRule#RelOptRule(org.eigenbase.relopt.RelOptRuleOperand, String)}
+ * <p><b>Note:</b> Rule descriptions are obtained via
+ * {@link org.apache.calcite.plan.RelOptRule#toString()}. By default they are
+ * the class's simple name (e.g. class name sans package), unless the class is
+ * an inner class, in which case the default is the inner class's simple
+ * name. Some rules explicitly provide alternate descriptions by calling the
+ * {@link org.apache.calcite.plan.RelOptRule#RelOptRule(org.apache.calcite.plan.RelOptRuleOperand, String)}
  * constructor.
  */
 public interface VolcanoPlannerPhaseRuleMappingInitializer {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoRelMetadataProvider.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoRelMetadataProvider.java b/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoRelMetadataProvider.java
index 98b0c7d..e898a61 100644
--- a/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoRelMetadataProvider.java
+++ b/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoRelMetadataProvider.java
@@ -14,10 +14,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.relopt.volcano;
+package org.apache.calcite.plan.volcano;
 
-import org.eigenbase.rel.*;
-import org.eigenbase.rel.metadata.*;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.metadata.Metadata;
+import org.apache.calcite.rel.metadata.RelMetadataProvider;
 
 import com.google.common.base.Function;
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 0446d79..273a4da 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
@@ -14,17 +14,26 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.relopt.volcano;
+package org.apache.calcite.plan.volcano;
 
-import java.util.*;
-import java.util.logging.*;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.util.*;
+import org.apache.calcite.plan.RelOptListener;
+import org.apache.calcite.plan.RelOptRuleCall;
+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 java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.logging.Level;
+
 /**
  * <code>VolcanoRuleCall</code> implements the {@link RelOptRuleCall} interface
  * for VolcanoPlanner.
@@ -104,8 +113,7 @@ public class VolcanoRuleCall extends RelOptRuleCall {
         // been registered. For now, let's make up something similar.
         String relDesc =
             "rel#" + rel.getId() + ":" + rel.getRelTypeName();
-        LOGGER.finest(
-            "call#" + id
+        LOGGER.finest("call#" + id
             + ": Rule " + getRule() + " arguments "
             + Arrays.toString(rels) + " created " + relDesc);
       }
@@ -154,8 +162,7 @@ public class VolcanoRuleCall extends RelOptRuleCall {
     try {
       if (volcanoPlanner.isRuleExcluded(getRule())) {
         if (LOGGER.isLoggable(Level.FINE)) {
-          LOGGER.fine(
-              "Rule [" + getRule() + "] not fired"
+          LOGGER.fine("Rule [" + getRule() + "] not fired"
               + " due to exclusion filter");
         }
         return;
@@ -189,8 +196,7 @@ public class VolcanoRuleCall extends RelOptRuleCall {
             volcanoPlanner.relImportances.get(rel);
         if ((importance != null) && (importance == 0d)) {
           if (LOGGER.isLoggable(Level.FINE)) {
-            LOGGER.fine(
-                "Rule [" + getRule() + "] not fired because"
+            LOGGER.fine("Rule [" + getRule() + "] not fired because"
                 + " operand #" + i + " (" + rel
                 + ") has importance=0");
           }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoRuleMatch.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoRuleMatch.java b/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoRuleMatch.java
index 58be603..bc120f8 100644
--- a/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoRuleMatch.java
+++ b/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoRuleMatch.java
@@ -14,10 +14,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.relopt.volcano;
+package org.apache.calcite.plan.volcano;
 
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
+import org.apache.calcite.plan.RelOptRuleOperand;
+import org.apache.calcite.plan.RelTrait;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
 
 /**
  * A match of a rule to a particular set of target relational expressions,


[58/58] [abbrv] incubator-calcite git commit: [CALCITE-370] Support GROUPING SETS, CUBE, ROLLUP in SQL and algebra

Posted by jh...@apache.org.
[CALCITE-370] Support GROUPING SETS, CUBE, ROLLUP in SQL and algebra

Includes support for parsing, validating, translating and executing (via EnumerableDefaults.groupByMultiple) GROUPING SETS, CUBE, ROLLUP.

The GROUPING function is now implemented yet. But the row type returned by Aggregate has optional indicator fields, one per key column, which will be used to implement them. Probably quite a few planner rules would find them confusing, so those planner rules should stick to simple Aggregates for now.


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

Branch: refs/heads/master
Commit: 6d79b5ebf7b1fa40455eb3df82998c58a8e08b8d
Parents: b9d8de3
Author: Julian Hyde <jh...@apache.org>
Authored: Thu Nov 13 19:21:22 2014 -0800
Committer: Julian Hyde <ju...@gmail.com>
Committed: Fri Nov 14 00:06:24 2014 -0800

----------------------------------------------------------------------
 core/src/main/codegen/templates/Parser.jj       |  65 +-
 .../adapter/enumerable/EnumerableAggregate.java |  85 ++-
 .../enumerable/EnumerableAggregateRule.java     |   2 +
 .../calcite/adapter/enumerable/PhysType.java    |  26 +
 .../adapter/enumerable/PhysTypeImpl.java        |  53 +-
 .../apache/calcite/adapter/jdbc/JdbcRules.java  |  15 +-
 .../org/apache/calcite/plan/RelOptRule.java     |  21 +-
 .../org/apache/calcite/plan/RelOptUtil.java     |  29 +-
 .../calcite/plan/SubstitutionVisitor.java       |  54 +-
 .../java/org/apache/calcite/rel/RelInput.java   |   4 +-
 .../org/apache/calcite/rel/core/Aggregate.java  | 177 +++--
 .../apache/calcite/rel/core/RelFactories.java   |  10 +-
 .../java/org/apache/calcite/rel/core/SetOp.java |   2 +-
 .../calcite/rel/externalize/RelJsonReader.java  |  22 +-
 .../calcite/rel/logical/LogicalAggregate.java   |  11 +-
 .../AggregateExpandDistinctAggregatesRule.java  |  30 +-
 .../rel/rules/AggregateFilterTransposeRule.java |   8 +-
 .../rel/rules/AggregateProjectMergeRule.java    |   6 +-
 .../AggregateProjectPullUpConstantsRule.java    |  17 +-
 .../rel/rules/AggregateReduceFunctionsRule.java |   6 +-
 .../rel/rules/AggregateStarTableRule.java       |   8 +-
 .../rel/rules/AggregateUnionAggregateRule.java  |  10 +-
 .../rel/rules/AggregateUnionTransposeRule.java  |  19 +-
 .../org/apache/calcite/sql/SqlCallBinding.java  |  12 +-
 .../apache/calcite/sql/SqlInternalOperator.java |  33 +-
 .../java/org/apache/calcite/sql/SqlKind.java    |  14 +-
 .../org/apache/calcite/sql/SqlOperator.java     |   2 +-
 .../apache/calcite/sql/SqlSpecialOperator.java  |   9 -
 .../sql/fun/SqlLiteralChainOperator.java        |   4 +-
 .../calcite/sql/fun/SqlStdOperatorTable.java    |  17 +-
 .../calcite/sql/fun/SqlThrowOperator.java       |   4 +-
 .../apache/calcite/sql/validate/AggChecker.java |  22 +-
 .../sql/validate/AggregatingSelectScope.java    |  25 +-
 .../calcite/sql/validate/SqlValidatorImpl.java  |  32 +-
 .../apache/calcite/sql2rel/RelDecorrelator.java |  30 +-
 .../apache/calcite/sql2rel/RelFieldTrimmer.java |  17 +-
 .../calcite/sql2rel/SqlToRelConverter.java      | 308 ++++++---
 .../main/java/org/apache/calcite/util/Bug.java  |   5 +
 .../org/apache/calcite/util/BuiltInMethod.java  |   4 +
 .../main/java/org/apache/calcite/util/Util.java |  17 +-
 .../apache/calcite/util/mapping/Mappings.java   |   5 +-
 .../org/apache/calcite/plan/RelWriterTest.java  |   5 +-
 .../calcite/sql/parser/SqlParserTest.java       | 133 +++-
 .../org/apache/calcite/test/FoodmartTest.java   |  10 +-
 .../java/org/apache/calcite/test/JdbcTest.java  |  24 +
 .../calcite/test/SqlToRelConverterTest.java     | 132 +++-
 .../apache/calcite/test/SqlValidatorTest.java   | 658 +++++++++----------
 .../calcite/test/SqlValidatorTestCase.java      |  64 +-
 .../calcite/test/SqlToRelConverterTest.xml      | 161 ++++-
 core/src/test/resources/sql/agg.oq              | 121 ++++
 .../calcite/linq4j/EnumerableDefaults.java      | 106 ++-
 .../java/org/apache/calcite/linq4j/Linq4j.java  |  17 +
 .../calcite/adapter/mongodb/MongoAggregate.java |  18 +-
 .../calcite/adapter/mongodb/MongoRules.java     |   2 +
 54 files changed, 1877 insertions(+), 814 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6d79b5eb/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 de98351..4776cdf 100644
--- a/core/src/main/codegen/templates/Parser.jj
+++ b/core/src/main/codegen/templates/Parser.jj
@@ -91,6 +91,7 @@ import org.apache.calcite.util.trace.CalciteTrace;
 import org.apache.calcite.util.DateTimeUtil;
 
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
 
 import java.io.Reader;
 import java.math.BigDecimal;
@@ -1883,25 +1884,69 @@ SqlNode WhereOpt() :
  */
 SqlNodeList GroupByOpt() :
 {
-    SqlNodeList list;
+    List<SqlNode> list = Lists.newArrayList();
     SqlNode e;
     SqlParserPos pos;
 }
 {
-    LOOKAHEAD(4)
     <GROUP> { pos = getPos(); }
-    <BY> list = ExpressionCommaList(pos, ExprContext.ACCEPT_SUBQUERY)
-    {
-        return list;
+    <BY> list = GroupingElementList() {
+        if (list.size() == 1
+            && list.get(0) instanceof SqlNodeList) {
+            // Preserve the old behavior that "GROUP BY ()" returns a list
+            // of length 0, not a of length 1 containing an empty list.
+            return (SqlNodeList) list.get(0);
+        }
+        return new SqlNodeList(list, pos);
     }
-    |
-    <GROUP> <BY> <LPAREN> <RPAREN>
+|
     {
+        return null;
+    }
+}
+
+List<SqlNode> GroupingElementList() :
+{
+    List<SqlNode> list = Lists.newArrayList();
+    SqlNode e;
+}
+{
+    e = GroupingElement() { list.add(e); }
+    (
+        <COMMA>
+        e = GroupingElement() { list.add(e); }
+    )*
+    { return list; }
+}
+
+SqlNode GroupingElement() :
+{
+    List<SqlNode> list;
+    SqlNodeList nlist;
+    SqlNode e;
+    SqlParserPos pos;
+}
+{
+    <GROUPING> { pos = getPos(); }
+    <SETS> <LPAREN> list = GroupingElementList() <RPAREN> {
+        return SqlStdOperatorTable.GROUPING_SETS.createCall(pos, list);
+    }
+|   <ROLLUP> { pos = getPos(); }
+    <LPAREN> nlist = ExpressionCommaList(pos, ExprContext.ACCEPT_SUBQUERY)
+    <RPAREN> {
+        return SqlStdOperatorTable.ROLLUP.createCall(nlist);
+    }
+|   <CUBE> { pos = getPos(); }
+    <LPAREN> nlist = ExpressionCommaList(pos, ExprContext.ACCEPT_SUBQUERY)
+    <RPAREN> {
+        return SqlStdOperatorTable.CUBE.createCall(nlist);
+    }
+|   LOOKAHEAD(3)
+    <LPAREN> <RPAREN> {
         return new SqlNodeList(getPos());
     }
-    |
-    {
-        return null;
+|   e = Expression(ExprContext.ACCEPT_SUBQUERY) {
+        return e;
     }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6d79b5eb/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableAggregate.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableAggregate.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableAggregate.java
index 91014bf..a6d3885 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableAggregate.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableAggregate.java
@@ -45,6 +45,7 @@ import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
 
 import java.lang.reflect.Type;
 import java.util.ArrayList;
@@ -66,10 +67,12 @@ public class EnumerableAggregate extends Aggregate
       RelOptCluster cluster,
       RelTraitSet traitSet,
       RelNode child,
+      boolean indicator,
       ImmutableBitSet groupSet,
+      List<ImmutableBitSet> groupSets,
       List<AggregateCall> aggCalls)
       throws InvalidRelException {
-    super(cluster, traitSet, child, groupSet, aggCalls);
+    super(cluster, traitSet, child, indicator, groupSet, groupSets, aggCalls);
     assert getConvention() instanceof EnumerableConvention;
 
     for (AggregateCall aggCall : aggCalls) {
@@ -86,11 +89,12 @@ public class EnumerableAggregate extends Aggregate
     }
   }
 
-  @Override public EnumerableAggregate copy(RelTraitSet traitSet,
-      RelNode input, ImmutableBitSet groupSet, List<AggregateCall> aggCalls) {
+  @Override public EnumerableAggregate copy(RelTraitSet traitSet, RelNode input,
+      boolean indicator, ImmutableBitSet groupSet,
+      List<ImmutableBitSet> groupSets, List<AggregateCall> aggCalls) {
     try {
-      return new EnumerableAggregate(getCluster(), traitSet, input,
-          groupSet, aggCalls);
+      return new EnumerableAggregate(getCluster(), traitSet, input, indicator,
+          groupSet, groupSets, aggCalls);
     } catch (InvalidRelException e) {
       // Semantic error not possible. Must be a bug. Convert to
       // internal error.
@@ -178,15 +182,9 @@ public class EnumerableAggregate extends Aggregate
         Expressions.parameter(inputPhysType.getJavaRowType(), "a0");
 
     final PhysType keyPhysType =
-        inputPhysType.project(groupSet.toList(), JavaRowFormat.LIST);
+        inputPhysType.project(groupSet.asList(), getGroupType() != Group.SIMPLE,
+            JavaRowFormat.LIST);
     final int keyArity = groupSet.cardinality();
-    final Expression keySelector =
-        builder.append(
-            "keySelector",
-            inputPhysType.generateSelector(
-                parameter,
-                groupSet.toList(),
-                keyPhysType.getFormat()));
 
     final List<AggImpState> aggs =
         new ArrayList<AggImpState>(aggCalls.size());
@@ -362,7 +360,34 @@ public class EnumerableAggregate extends Aggregate
               new AggResultContextImpl(resultBlock, agg.state)));
     }
     resultBlock.add(physType.record(results));
-    if (keyArity == 0) {
+    if (getGroupType() != Group.SIMPLE) {
+      final List<Expression> list = Lists.newArrayList();
+      for (ImmutableBitSet set : groupSets) {
+        list.add(
+            inputPhysType.generateSelector(parameter, groupSet.toList(),
+                set.toList(), keyPhysType.getFormat()));
+      }
+      final Expression keySelectors_ =
+          builder.append("keySelectors",
+              Expressions.call(BuiltInMethod.ARRAYS_AS_LIST.method,
+                  list));
+      final Expression resultSelector =
+          builder.append("resultSelector",
+              Expressions.lambda(Function2.class,
+                  resultBlock.toBlock(),
+                  key_,
+                  acc_));
+      builder.add(
+          Expressions.return_(null,
+              Expressions.call(
+                  BuiltInMethod.GROUP_BY_MULTIPLE.method,
+                  Expressions.list(childExp,
+                      keySelectors_,
+                      accumulatorInitializer,
+                      accumulatorAdder,
+                      resultSelector)
+                      .appendIfNotNull(keyPhysType.comparer()))));
+    } else if (keyArity == 0) {
       final Expression resultSelector =
           builder.append(
               "resultSelector",
@@ -393,28 +418,26 @@ public class EnumerableAggregate extends Aggregate
                   Expressions.<Expression>list()
                       .appendIfNotNull(physType.comparer()))));
     } else {
-      final Expression resultSelector =
-          builder.append(
-              "resultSelector",
-              Expressions.lambda(
-                  Function2.class,
+      final Expression keySelector_ =
+          builder.append("keySelector",
+              inputPhysType.generateSelector(parameter,
+                  groupSet.toList(),
+                  keyPhysType.getFormat()));
+      final Expression resultSelector_ =
+          builder.append("resultSelector",
+              Expressions.lambda(Function2.class,
                   resultBlock.toBlock(),
                   key_,
                   acc_));
       builder.add(
-          Expressions.return_(
-              null,
-              Expressions.call(
-                  childExp,
+          Expressions.return_(null,
+              Expressions.call(childExp,
                   BuiltInMethod.GROUP_BY2.method,
-                  Expressions
-                      .list(
-                          keySelector,
-                          accumulatorInitializer,
-                          accumulatorAdder,
-                          resultSelector)
-                      .appendIfNotNull(
-                          keyPhysType.comparer()))));
+                  Expressions.list(keySelector_,
+                      accumulatorInitializer,
+                      accumulatorAdder,
+                      resultSelector_)
+                      .appendIfNotNull(keyPhysType.comparer()))));
     }
     return implementor.result(physType, builder.toBlock());
   }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6d79b5eb/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableAggregateRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableAggregateRule.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableAggregateRule.java
index c04cd03..340801f 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableAggregateRule.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableAggregateRule.java
@@ -42,7 +42,9 @@ class EnumerableAggregateRule extends ConverterRule {
           rel.getCluster(),
           traitSet,
           convert(agg.getInput(), traitSet),
+          agg.indicator,
           agg.getGroupSet(),
+          agg.getGroupSets(),
           agg.getAggCallList());
     } catch (InvalidRelException e) {
       EnumerableRules.LOGGER.fine(e.toString());

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6d79b5eb/core/src/main/java/org/apache/calcite/adapter/enumerable/PhysType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/PhysType.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/PhysType.java
index e620c13..dd6d2b2 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/PhysType.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/PhysType.java
@@ -118,6 +118,22 @@ public interface PhysType {
       List<Integer> fields,
       JavaRowFormat targetFormat);
 
+  /** Generates a lambda expression that is a selector for the given fields from
+   * an expression.
+   *
+   * <p>{@code usedFields} must be a subset of {@code fields}.
+   * For each field, there is a corresponding indicator field.
+   * If a field is used, its value is assigned and its indicator is left
+   * {@code false}.
+   * If a field is not used, its value is not assigned and its indicator is
+   * set to {@code true};
+   * This will become a value of 1 when {@code GROUPING(field)} is called. */
+  Expression generateSelector(
+      ParameterExpression parameter,
+      List<Integer> fields,
+      List<Integer> usedFields,
+      JavaRowFormat targetFormat);
+
   /** Generates a selector for the given fields from an expression. */
   Expression selector(
       ParameterExpression parameter,
@@ -131,6 +147,16 @@ public interface PhysType {
       List<Integer> integers,
       JavaRowFormat format);
 
+  /** Projects a given collection of fields from this input record, optionally
+   * with indicator fields, into a particular preferred output format.
+   *
+   * <p>The output format is optimized if there are 0 or 1 fields
+   * and indicators are disabled. */
+  PhysType project(
+      List<Integer> integers,
+      boolean indicator,
+      JavaRowFormat format);
+
   /** Returns a lambda to create a collation key and a comparator. The
    * comparator is sometimes null. */
   Pair<Expression, Expression> generateCollationKey(

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6d79b5eb/core/src/main/java/org/apache/calcite/adapter/enumerable/PhysTypeImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/PhysTypeImpl.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/PhysTypeImpl.java
index 562e4ba..1a9122a 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/PhysTypeImpl.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/PhysTypeImpl.java
@@ -31,11 +31,11 @@ import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.runtime.Utilities;
+import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.util.BuiltInMethod;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Util;
 
-import com.google.common.base.Function;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 
@@ -104,14 +104,25 @@ public class PhysTypeImpl implements PhysType {
   }
 
   public PhysType project(List<Integer> integers, JavaRowFormat format) {
-    RelDataType projectedRowType =
-        typeFactory.createStructType(
-            Lists.transform(integers,
-                new Function<Integer, RelDataTypeField>() {
-                  public RelDataTypeField apply(Integer index) {
-                    return rowType.getFieldList().get(index);
-                  }
-                }));
+    return project(integers, false, format);
+  }
+
+  public PhysType project(List<Integer> integers, boolean indicator,
+      JavaRowFormat format) {
+    final RelDataTypeFactory.FieldInfoBuilder builder = typeFactory.builder();
+    for (int index : integers) {
+      builder.add(rowType.getFieldList().get(index));
+    }
+    if (indicator) {
+      final RelDataType booleanType =
+          typeFactory.createTypeWithNullability(
+              typeFactory.createSqlType(SqlTypeName.BOOLEAN), false);
+      for (int index : integers) {
+        builder.add("i$" + rowType.getFieldList().get(index).getName(),
+            booleanType);
+      }
+    }
+    RelDataType projectedRowType = builder.build();
     return of(typeFactory, projectedRowType, format.optimize(projectedRowType));
   }
 
@@ -145,6 +156,30 @@ public class PhysTypeImpl implements PhysType {
     }
   }
 
+  public Expression generateSelector(final ParameterExpression parameter,
+      final List<Integer> fields, List<Integer> usedFields,
+      JavaRowFormat targetFormat) {
+    final PhysType targetPhysType =
+        project(fields, true, targetFormat);
+    final List<Expression> expressions = Lists.newArrayList();
+    for (Integer field : fields) {
+      if (usedFields.contains(field)) {
+        expressions.add(fieldReference(parameter, field));
+      } else {
+        final Primitive primitive =
+            Primitive.of(targetPhysType.fieldClass(field));
+        expressions.add(
+            Expressions.constant(
+                primitive != null ? primitive.defaultValue : null));
+      }
+    }
+    for (Integer field : fields) {
+      expressions.add(Expressions.constant(!usedFields.contains(field)));
+    }
+    return Expressions.lambda(Function1.class,
+        targetPhysType.record(expressions), parameter);
+  }
+
   public Expression selector(
       ParameterExpression parameter,
       List<Integer> fields,

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6d79b5eb/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcRules.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcRules.java b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcRules.java
index a085429..eb5ff58 100644
--- a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcRules.java
+++ b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcRules.java
@@ -550,8 +550,8 @@ public class JdbcRules {
           agg.getTraitSet().replace(out);
       try {
         return new JdbcAggregate(rel.getCluster(), traitSet,
-            convert(agg.getInput(), traitSet), agg.getGroupSet(),
-            agg.getAggCallList());
+            convert(agg.getInput(), traitSet), agg.indicator, agg.getGroupSet(),
+            agg.getGroupSets(), agg.getAggCallList());
       } catch (InvalidRelException e) {
         LOGGER.fine(e.toString());
         return null;
@@ -565,18 +565,21 @@ public class JdbcRules {
         RelOptCluster cluster,
         RelTraitSet traitSet,
         RelNode child,
+        boolean indicator,
         ImmutableBitSet groupSet,
+        List<ImmutableBitSet> groupSets,
         List<AggregateCall> aggCalls)
         throws InvalidRelException {
-      super(cluster, traitSet, child, groupSet, aggCalls);
+      super(cluster, traitSet, child, indicator, groupSet, groupSets, aggCalls);
       assert getConvention() instanceof JdbcConvention;
     }
 
     @Override public JdbcAggregate copy(RelTraitSet traitSet, RelNode input,
-        ImmutableBitSet groupSet, List<AggregateCall> aggCalls) {
+        boolean indicator, ImmutableBitSet groupSet,
+        List<ImmutableBitSet> groupSets, List<AggregateCall> aggCalls) {
       try {
-        return new JdbcAggregate(getCluster(), traitSet, input, groupSet,
-            aggCalls);
+        return new JdbcAggregate(getCluster(), traitSet, input, indicator,
+            groupSet, groupSets, aggCalls);
       } catch (InvalidRelException e) {
         // Semantic error not possible. Must be a bug. Convert to
         // internal error.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6d79b5eb/core/src/main/java/org/apache/calcite/plan/RelOptRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/RelOptRule.java b/core/src/main/java/org/apache/calcite/plan/RelOptRule.java
index 193ca9b..d9d2e6f 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptRule.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptRule.java
@@ -118,8 +118,8 @@ public abstract class RelOptRule {
   }
 
   /**
-   * Creates an operand that matches a relational expression that has no
-   * children.
+   * Creates an operand that matches a relational expression that has a
+   * particular trait and predicate.
    *
    * @param clazz Class of relational expression to match (must not be null)
    * @param trait Trait to match, or null to match any trait
@@ -134,6 +134,23 @@ public abstract class RelOptRule {
   }
 
   /**
+   * Creates an operand that matches a relational expression that has no
+   * children.
+   *
+   * @param clazz Class of relational expression to match (must not be null)
+   * @param trait Trait to match, or null to match any trait
+   * @return Operand
+   */
+  public static <R extends RelNode> RelOptRuleOperand operand(
+      Class<R> clazz,
+      RelTrait trait,
+      Predicate<? super R> predicate,
+      RelOptRuleOperand first,
+      RelOptRuleOperand... rest) {
+    return new RelOptRuleOperand(clazz, trait, predicate, some(first, rest));
+  }
+
+  /**
    * Creates an operand that matches a relational expression with a given
    * list of children.
    *

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6d79b5eb/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 5949dbf..f043c66 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
@@ -357,11 +357,8 @@ public abstract class RelOptUtil {
               extraName);
 
       ret =
-          new LogicalAggregate(
-              ret.getCluster(),
-              ret,
-              ImmutableBitSet.of(),
-              ImmutableList.of(aggCall));
+          new LogicalAggregate(ret.getCluster(), ret, false,
+              ImmutableBitSet.of(), null, ImmutableList.of(aggCall));
     }
 
     return ret;
@@ -401,7 +398,8 @@ public abstract class RelOptUtil {
       final int keyCount = ret.getRowType().getFieldCount();
       if (!needsOuterJoin) {
         return Pair.<RelNode, Boolean>of(
-            new LogicalAggregate(cluster, ret, ImmutableBitSet.range(keyCount),
+            new LogicalAggregate(cluster, ret, false,
+                ImmutableBitSet.range(keyCount), null,
                 ImmutableList.<AggregateCall>of()),
             false);
       }
@@ -435,10 +433,8 @@ public abstract class RelOptUtil {
               null,
               null);
 
-      ret = new LogicalAggregate(
-          cluster,
-          ret,
-          ImmutableBitSet.range(projectedKeyCount),
+      ret = new LogicalAggregate(cluster, ret, false,
+          ImmutableBitSet.range(projectedKeyCount), null,
           ImmutableList.of(aggCall));
 
       switch (logic) {
@@ -680,11 +676,8 @@ public abstract class RelOptUtil {
               null));
     }
 
-    return new LogicalAggregate(
-        rel.getCluster(),
-        rel,
-        ImmutableBitSet.of(),
-        aggCalls);
+    return new LogicalAggregate(rel.getCluster(), rel, false,
+        ImmutableBitSet.of(), null, aggCalls);
   }
 
   /**
@@ -695,10 +688,8 @@ public abstract class RelOptUtil {
    * @return rel implementing DISTINCT
    */
   public static RelNode createDistinctRel(RelNode rel) {
-    return new LogicalAggregate(
-        rel.getCluster(),
-        rel,
-        ImmutableBitSet.range(rel.getRowType().getFieldCount()),
+    return new LogicalAggregate(rel.getCluster(), rel, false,
+        ImmutableBitSet.range(rel.getRowType().getFieldCount()), null,
         ImmutableList.<AggregateCall>of());
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6d79b5eb/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 c61208e..b6b1c65 100644
--- a/core/src/main/java/org/apache/calcite/plan/SubstitutionVisitor.java
+++ b/core/src/main/java/org/apache/calcite/plan/SubstitutionVisitor.java
@@ -48,6 +48,7 @@ import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.validate.SqlValidatorUtil;
+import org.apache.calcite.util.Bug;
 import org.apache.calcite.util.ControlFlowException;
 import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.IntList;
@@ -64,6 +65,7 @@ import com.google.common.base.Objects;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Predicate;
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
 import com.google.common.collect.LinkedHashMultimap;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
@@ -223,7 +225,8 @@ public class SubstitutionVisitor {
     if (rel instanceof Aggregate) {
       final Aggregate aggregate = (Aggregate) rel;
       final MutableRel input = toMutable(aggregate.getInput());
-      return MutableAggregate.of(input, aggregate.getGroupSet(),
+      return MutableAggregate.of(input, aggregate.indicator,
+          aggregate.getGroupSet(), aggregate.getGroupSets(),
           aggregate.getAggCallList());
     }
     throw new RuntimeException("cannot translate " + rel + " to MutableRel");
@@ -598,7 +601,8 @@ public class SubstitutionVisitor {
     case AGGREGATE:
       final MutableAggregate aggregate = (MutableAggregate) node;
       return new LogicalAggregate(node.cluster, fromMutable(aggregate.input),
-          aggregate.groupSet, aggregate.aggCalls);
+          aggregate.indicator, aggregate.groupSet, aggregate.groupSets,
+          aggregate.aggCalls);
     case SORT:
       final MutableSort sort = (MutableSort) node;
       return new Sort(node.cluster, node.cluster.traitSetOf(sort.collation),
@@ -1145,11 +1149,20 @@ public class SubstitutionVisitor {
   }
 
   public static MutableAggregate permute(MutableAggregate aggregate,
-      MutableRel input, Mapping mapping) {
+      MutableRel input, final Mapping mapping) {
     ImmutableBitSet groupSet = Mappings.apply(mapping, aggregate.getGroupSet());
+    ImmutableList<ImmutableBitSet> groupSets =
+        ImmutableList.copyOf(
+            Iterables.transform(aggregate.getGroupSets(),
+                new Function<ImmutableBitSet, ImmutableBitSet>() {
+                  public ImmutableBitSet apply(ImmutableBitSet input1) {
+                    return Mappings.apply(mapping, input1);
+                  }
+                }));
     List<AggregateCall> aggregateCalls =
         apply(mapping, aggregate.getAggCallList());
-    return MutableAggregate.of(input, groupSet, aggregateCalls);
+    return MutableAggregate.of(input, aggregate.indicator, groupSet, groupSets,
+        aggregateCalls);
   }
 
   private static List<AggregateCall> apply(final Mapping mapping,
@@ -1164,6 +1177,10 @@ public class SubstitutionVisitor {
 
   public static MutableRel unifyAggregates(MutableAggregate query,
       MutableAggregate target) {
+    if (query.getGroupType() != Aggregate.Group.SIMPLE
+        || target.getGroupType() != Aggregate.Group.SIMPLE) {
+      throw new AssertionError(Bug.CALCITE_461_FIXED);
+    }
     MutableRel result;
     if (query.getGroupSet().equals(target.getGroupSet())) {
       // Same level of aggregation. Generate a project.
@@ -1206,7 +1223,8 @@ public class SubstitutionVisitor {
                 ImmutableList.of(target.groupSet.cardinality() + i),
                 aggregateCall.type, aggregateCall.name));
       }
-      result = MutableAggregate.of(target, groupSet.build(), aggregateCalls);
+      result = MutableAggregate.of(target, false, groupSet.build(), null,
+          aggregateCalls);
     }
     return MutableRels.createCastRel(result, query.getRowType(), true);
   }
@@ -1620,22 +1638,31 @@ public class SubstitutionVisitor {
   /** Mutable equivalent of
    * {@link org.apache.calcite.rel.logical.LogicalAggregate}. */
   private static class MutableAggregate extends MutableSingleRel {
+    public final boolean indicator;
     private final ImmutableBitSet groupSet;
+    private final ImmutableList<ImmutableBitSet> groupSets;
     private final List<AggregateCall> aggCalls;
 
     private MutableAggregate(MutableRel input, RelDataType rowType,
-        ImmutableBitSet groupSet, List<AggregateCall> aggCalls) {
+        boolean indicator, ImmutableBitSet groupSet,
+        List<ImmutableBitSet> groupSets, List<AggregateCall> aggCalls) {
       super(MutableRelType.AGGREGATE, rowType, input);
+      this.indicator = indicator;
       this.groupSet = groupSet;
+      this.groupSets = groupSets == null
+          ? ImmutableList.of(groupSet)
+          : ImmutableList.copyOf(groupSets);
       this.aggCalls = aggCalls;
     }
 
-    static MutableAggregate of(MutableRel input, ImmutableBitSet groupSet,
+    static MutableAggregate of(MutableRel input, boolean indicator,
+        ImmutableBitSet groupSet, ImmutableList<ImmutableBitSet> groupSets,
         List<AggregateCall> aggCalls) {
       RelDataType rowType =
           Aggregate.deriveRowType(input.cluster.getTypeFactory(),
-              input.getRowType(), groupSet, aggCalls);
-      return new MutableAggregate(input, rowType, groupSet, aggCalls);
+              input.getRowType(), indicator, groupSet, groupSets, aggCalls);
+      return new MutableAggregate(input, rowType, indicator, groupSet,
+          groupSets, aggCalls);
     }
 
     @Override public boolean equals(Object obj) {
@@ -1652,6 +1679,7 @@ public class SubstitutionVisitor {
 
     @Override public StringBuilder digest(StringBuilder buf) {
       return buf.append("Aggregate(groupSet: ").append(groupSet)
+          .append(", groupSets: ").append(groupSets)
           .append(", calls: ").append(aggCalls).append(")");
     }
 
@@ -1659,9 +1687,17 @@ public class SubstitutionVisitor {
       return groupSet;
     }
 
+    public ImmutableList<ImmutableBitSet> getGroupSets() {
+      return groupSets;
+    }
+
     public List<AggregateCall> getAggCallList() {
       return aggCalls;
     }
+
+    public Aggregate.Group getGroupType() {
+      return Aggregate.Group.induce(groupSet, groupSets);
+    }
   }
 
   /** Mutable equivalent of {@link org.apache.calcite.rel.core.Sort}. */

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6d79b5eb/core/src/main/java/org/apache/calcite/rel/RelInput.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/RelInput.java b/core/src/main/java/org/apache/calcite/rel/RelInput.java
index 2ffe64f..38be893 100644
--- a/core/src/main/java/org/apache/calcite/rel/RelInput.java
+++ b/core/src/main/java/org/apache/calcite/rel/RelInput.java
@@ -53,6 +53,8 @@ public interface RelInput {
 
   ImmutableBitSet getBitSet(String tag);
 
+  List<ImmutableBitSet> getBitSetList(String tag);
+
   List<AggregateCall> getAggregateCalls(String tag);
 
   Object get(String tag);
@@ -82,7 +84,7 @@ public interface RelInput {
 
   List<List<RexLiteral>> getTuples(String tag);
 
-  boolean getBoolean(String tag);
+  boolean getBoolean(String tag, boolean default_);
 }
 
 // End RelInput.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6d79b5eb/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 d0e19be..c8f165c 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
@@ -30,22 +30,22 @@ import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeField;
-import org.apache.calcite.rel.type.RelDataTypeFieldImpl;
 import org.apache.calcite.runtime.CalciteException;
 import org.apache.calcite.runtime.Resources;
 import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.SqlOperatorBinding;
 import org.apache.calcite.sql.SqlUtil;
 import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.validate.SqlValidatorException;
-import org.apache.calcite.util.CompositeList;
 import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.IntList;
 import org.apache.calcite.util.Util;
 
+import com.google.common.base.Preconditions;
+import com.google.common.base.Predicate;
 import com.google.common.collect.ImmutableList;
 
-import java.util.AbstractList;
 import java.util.List;
 
 /**
@@ -65,33 +65,75 @@ import java.util.List;
  * </ul>
  */
 public abstract class Aggregate extends SingleRel {
+  /**
+   * @see org.apache.calcite.util.Bug#CALCITE_461_FIXED
+   */
+  public static final Predicate<Aggregate> IS_SIMPLE =
+      new Predicate<Aggregate>() {
+        public boolean apply(Aggregate input) {
+          return input.getGroupType() == Group.SIMPLE;
+        }
+      };
+
   //~ Instance fields --------------------------------------------------------
 
+  public final boolean indicator;
   protected final List<AggregateCall> aggCalls;
   protected final ImmutableBitSet groupSet;
+  public final ImmutableList<ImmutableBitSet> groupSets;
 
   //~ Constructors -----------------------------------------------------------
 
   /**
    * Creates an Aggregate.
    *
+   * <p>All members of {@code groupSets} must be sub-sets of {@code groupSet}.
+   * For a simple {@code GROUP BY}, {@code groupSets} is a singleton list
+   * containing {@code groupSet}.
+   *
+   * <p>If {@code GROUP BY} is not specified,
+   * or equivalently if {@code GROUP BY ()} is specified,
+   * {@code groupSet} will be the empty set,
+   * and {@code groupSets} will have one element, that empty set.
+   *
+   * <p>If {@code CUBE}, {@code ROLLUP} or {@code GROUPING SETS} are
+   * specified, {@code groupSets} will have additional elements,
+   * but they must each be a subset of {@code groupSet},
+   * and they must be sorted by inclusion:
+   * {@code (0, 1, 2), (1), (0, 2), (0), ()}.
+   *
    * @param cluster  Cluster
    * @param traits   Traits
    * @param child    Child
+   * @param indicator Whether row type should include indicator fields to
+   *                 indicate which grouping set is active; must be true if
+   *                 aggregate is not simple
    * @param groupSet Bit set of grouping fields
+   * @param groupSets List of all grouping sets; null for just {@code groupSet}
    * @param aggCalls Collection of calls to aggregate functions
    */
   protected Aggregate(
       RelOptCluster cluster,
       RelTraitSet traits,
       RelNode child,
+      boolean indicator,
       ImmutableBitSet groupSet,
+      List<ImmutableBitSet> groupSets,
       List<AggregateCall> aggCalls) {
     super(cluster, traits, child);
-    assert aggCalls != null;
+    this.indicator = indicator;
     this.aggCalls = ImmutableList.copyOf(aggCalls);
-    this.groupSet = groupSet;
-    assert groupSet != null;
+    this.groupSet = Preconditions.checkNotNull(groupSet);
+    if (groupSets == null) {
+      this.groupSets = ImmutableList.of(groupSet);
+    } else {
+      this.groupSets = ImmutableList.copyOf(groupSets);
+      assert Util.isStrictlySorted(groupSets, ImmutableBitSet.COMPARATOR)
+          : groupSets;
+      for (ImmutableBitSet set : groupSets) {
+        assert groupSet.contains(set);
+      }
+    }
     assert groupSet.length() <= child.getRowType().getFieldCount();
     for (AggregateCall aggCall : aggCalls) {
       assert typeMatchesInferred(aggCall, true);
@@ -103,14 +145,17 @@ public abstract class Aggregate extends SingleRel {
    */
   protected Aggregate(RelInput input) {
     this(input.getCluster(), input.getTraitSet(), input.getInput(),
-        input.getBitSet("group"), input.getAggregateCalls("aggs"));
+        input.getBoolean("indicator", false),
+        input.getBitSet("group"), input.getBitSetList("groups"),
+        input.getAggregateCalls("aggs"));
   }
 
   //~ Methods ----------------------------------------------------------------
 
   @Override public final RelNode copy(RelTraitSet traitSet,
       List<RelNode> inputs) {
-    return copy(traitSet, sole(inputs), groupSet, aggCalls);
+    return copy(traitSet, sole(inputs), indicator, groupSet, groupSets,
+        aggCalls);
   }
 
   /** Creates a copy of this aggregate.
@@ -118,7 +163,8 @@ public abstract class Aggregate extends SingleRel {
    * @see #copy(org.apache.calcite.plan.RelTraitSet, java.util.List)
    */
   public abstract Aggregate copy(RelTraitSet traitSet, RelNode input,
-      ImmutableBitSet groupSet, List<AggregateCall> aggCalls);
+      boolean indicator, ImmutableBitSet groupSet,
+      List<ImmutableBitSet> groupSets, List<AggregateCall> aggCalls);
 
   // implement RelNode
   public boolean isDistinct() {
@@ -160,9 +206,19 @@ public abstract class Aggregate extends SingleRel {
     return groupSet;
   }
 
+  /**
+   * Returns the list of grouping sets computed by this Aggregate.
+   */
+  public ImmutableList<ImmutableBitSet> getGroupSets() {
+    return groupSets;
+  }
+
   public RelWriter explainTerms(RelWriter pw) {
+    // We skip the "groups" element if it is a singleton of "group".
     super.explainTerms(pw)
+        .itemIf("indicator", indicator, indicator)
         .item("group", groupSet)
+        .itemIf("groups", groupSets, getGroupType() != Group.SIMPLE)
         .itemIf("aggs", aggCalls, pw.nest());
     if (!pw.nest()) {
       for (Ord<AggregateCall> ord : Ord.zip(aggCalls)) {
@@ -172,8 +228,7 @@ public abstract class Aggregate extends SingleRel {
     return pw;
   }
 
-  // implement RelNode
-  public double getRows() {
+  @Override public double getRows() {
     // Assume that each sort column has 50% of the value count.
     // Therefore one sort column has .5 * rowCount,
     // 2 sort columns give .75 * rowCount.
@@ -188,7 +243,7 @@ public abstract class Aggregate extends SingleRel {
     }
   }
 
-  public RelOptCost computeSelfCost(RelOptPlanner planner) {
+  @Override public RelOptCost computeSelfCost(RelOptPlanner planner) {
     // REVIEW jvs 24-Aug-2008:  This is bogus, but no more bogus
     // than what's currently in Join.
     double rowCount = RelMetadataQuery.getRowCount(this);
@@ -196,50 +251,53 @@ public abstract class Aggregate extends SingleRel {
   }
 
   protected RelDataType deriveRowType() {
-    return deriveRowType(
-        getCluster().getTypeFactory(),
-        getInput().getRowType(),
-        groupSet,
-        aggCalls);
+    return deriveRowType(getCluster().getTypeFactory(), getInput().getRowType(),
+        indicator, groupSet, groupSets, aggCalls);
   }
 
   /** Computes the row type of an {@code Aggregate} before it exists. */
   public static RelDataType deriveRowType(RelDataTypeFactory typeFactory,
-      final RelDataType inputRowType, ImmutableBitSet groupSet,
+      final RelDataType inputRowType, boolean indicator,
+      ImmutableBitSet groupSet, List<ImmutableBitSet> groupSets,
       final List<AggregateCall> aggCalls) {
     final IntList groupList = groupSet.toList();
     assert groupList.size() == groupSet.cardinality();
-    return typeFactory.createStructType(
-        CompositeList.of(
-            // fields derived from grouping columns
-            new AbstractList<RelDataTypeField>() {
-              public int size() {
-                return groupList.size();
-              }
-
-              public RelDataTypeField get(int index) {
-                return inputRowType.getFieldList().get(
-                    groupList.get(index));
-              }
-            },
-
-            // fields derived from aggregate calls
-            new AbstractList<RelDataTypeField>() {
-              public int size() {
-                return aggCalls.size();
-              }
-
-              public RelDataTypeField get(int index) {
-                final AggregateCall aggCall = aggCalls.get(index);
-                String name;
-                if (aggCall.name != null) {
-                  name = aggCall.name;
-                } else {
-                  name = "$f" + (groupList.size() + index);
-                }
-                return new RelDataTypeFieldImpl(name, index, aggCall.type);
-              }
-            }));
+    final RelDataTypeFactory.FieldInfoBuilder builder = typeFactory.builder();
+    for (int groupKey : groupList) {
+      final RelDataTypeField field = inputRowType.getFieldList().get(groupKey);
+      boolean nullable = field.getType().isNullable()
+          || indicator
+          && !allContain(groupSets, groupKey);
+      builder.add(field).nullable(nullable);
+    }
+    if (indicator) {
+      for (int groupKey : groupList) {
+        final RelDataType booleanType =
+            typeFactory.createTypeWithNullability(
+                typeFactory.createSqlType(SqlTypeName.BOOLEAN), false);
+        builder.add("i$" + inputRowType.getFieldList().get(groupKey),
+            booleanType);
+      }
+    }
+    for (Ord<AggregateCall> aggCall : Ord.zip(aggCalls)) {
+      String name;
+      if (aggCall.e.name != null) {
+        name = aggCall.e.name;
+      } else {
+        name = "$f" + (groupList.size() + aggCall.i);
+      }
+      builder.add(name, aggCall.e.type);
+    }
+    return builder.build();
+  }
+
+  private static boolean allContain(List<ImmutableBitSet> bitSets, int bit) {
+    for (ImmutableBitSet bitSet : bitSets) {
+      if (!bitSet.get(bit)) {
+        return false;
+      }
+    }
+    return true;
   }
 
   /**
@@ -277,6 +335,27 @@ public abstract class Aggregate extends SingleRel {
     return false;
   }
 
+  /** Returns the type of roll-up. */
+  public Group getGroupType() {
+    return Group.induce(groupSet, groupSets);
+  }
+
+  /** What kind of roll-up is it? */
+  public enum Group {
+    SIMPLE,
+    ROLLUP,
+    CUBE,
+    OTHER;
+
+    public static Group induce(ImmutableBitSet groupSet,
+        List<ImmutableBitSet> groupSets) {
+      if (groupSets.size() == 1 && groupSets.get(0).equals(groupSet)) {
+        return SIMPLE;
+      }
+      return OTHER;
+    }
+  }
+
   //~ Inner Classes ----------------------------------------------------------
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6d79b5eb/core/src/main/java/org/apache/calcite/rel/core/RelFactories.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/RelFactories.java b/core/src/main/java/org/apache/calcite/rel/core/RelFactories.java
index 332fdd5..b975daf 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/RelFactories.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/RelFactories.java
@@ -148,7 +148,8 @@ public class RelFactories {
    */
   public interface AggregateFactory {
     /** Creates an aggregate. */
-    RelNode createAggregate(RelNode child, ImmutableBitSet groupSet,
+    RelNode createAggregate(RelNode child, boolean indicator,
+        ImmutableBitSet groupSet, ImmutableList<ImmutableBitSet> groupSets,
         List<AggregateCall> aggCalls);
   }
 
@@ -157,10 +158,11 @@ public class RelFactories {
    * that returns a vanilla {@link LogicalAggregate}.
    */
   private static class AggregateFactoryImpl implements AggregateFactory {
-    public RelNode createAggregate(RelNode child, ImmutableBitSet groupSet,
+    public RelNode createAggregate(RelNode child, boolean indicator,
+        ImmutableBitSet groupSet, ImmutableList<ImmutableBitSet> groupSets,
         List<AggregateCall> aggCalls) {
-      return new LogicalAggregate(child.getCluster(), child, groupSet,
-          aggCalls);
+      return new LogicalAggregate(child.getCluster(), child, indicator,
+          groupSet, groupSets, aggCalls);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6d79b5eb/core/src/main/java/org/apache/calcite/rel/core/SetOp.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/SetOp.java b/core/src/main/java/org/apache/calcite/rel/core/SetOp.java
index 2b17733..1c464d6 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/SetOp.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/SetOp.java
@@ -67,7 +67,7 @@ public abstract class SetOp extends AbstractRelNode {
    */
   protected SetOp(RelInput input) {
     this(input.getCluster(), input.getTraitSet(), input.getInputs(),
-        SqlKind.UNION, input.getBoolean("all"));
+        SqlKind.UNION, input.getBoolean("all", false));
   }
 
   //~ Methods ----------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6d79b5eb/core/src/main/java/org/apache/calcite/rel/externalize/RelJsonReader.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/externalize/RelJsonReader.java b/core/src/main/java/org/apache/calcite/rel/externalize/RelJsonReader.java
index 31dbdac..32ea764 100644
--- a/core/src/main/java/org/apache/calcite/rel/externalize/RelJsonReader.java
+++ b/core/src/main/java/org/apache/calcite/rel/externalize/RelJsonReader.java
@@ -134,10 +134,27 @@ public class RelJsonReader {
         return ImmutableBitSet.of(getIntegerList(tag));
       }
 
+      public List<ImmutableBitSet> getBitSetList(String tag) {
+        List<List<Integer>> list = getIntegerListList(tag);
+        if (list == null) {
+          return null;
+        }
+        final ImmutableList.Builder<ImmutableBitSet> builder =
+            ImmutableList.builder();
+        for (List<Integer> integers : list) {
+          builder.add(ImmutableBitSet.of(integers));
+        }
+        return builder.build();
+      }
+
       public List<Integer> getIntegerList(String tag) {
         return (List<Integer>) jsonRel.get(tag);
       }
 
+      public List<List<Integer>> getIntegerListList(String tag) {
+        return (List<List<Integer>>) jsonRel.get(tag);
+      }
+
       public List<AggregateCall> getAggregateCalls(String tag) {
         List<Map<String, Object>> jsonAggs = (List) jsonRel.get(tag);
         final List<AggregateCall> inputs = new ArrayList<AggregateCall>();
@@ -159,8 +176,9 @@ public class RelJsonReader {
         return ((Number) jsonRel.get(tag)).floatValue();
       }
 
-      public boolean getBoolean(String tag) {
-        return (Boolean) jsonRel.get(tag);
+      public boolean getBoolean(String tag, boolean default_) {
+        final Boolean b = (Boolean) jsonRel.get(tag);
+        return b != null ? b : default_;
       }
 
       public <E extends Enum<E>> E getEnum(String tag, Class<E> enumClass) {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6d79b5eb/core/src/main/java/org/apache/calcite/rel/logical/LogicalAggregate.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/logical/LogicalAggregate.java b/core/src/main/java/org/apache/calcite/rel/logical/LogicalAggregate.java
index 5f1d579..dd3e509 100644
--- a/core/src/main/java/org/apache/calcite/rel/logical/LogicalAggregate.java
+++ b/core/src/main/java/org/apache/calcite/rel/logical/LogicalAggregate.java
@@ -49,18 +49,23 @@ public final class LogicalAggregate extends Aggregate {
    * @param cluster  Cluster that this relational expression belongs to
    * @param child    input relational expression
    * @param groupSet Bit set of grouping fields
+   * @param groupSets Grouping sets, or null to use just {@code groupSet}
    * @param aggCalls Array of aggregates to compute, not null
    */
   public LogicalAggregate(
       RelOptCluster cluster,
       RelNode child,
+      boolean indicator,
       ImmutableBitSet groupSet,
+      List<ImmutableBitSet> groupSets,
       List<AggregateCall> aggCalls) {
     super(
         cluster,
         cluster.traitSetOf(Convention.NONE),
         child,
+        indicator,
         groupSet,
+        groupSets,
         aggCalls);
   }
 
@@ -74,9 +79,11 @@ public final class LogicalAggregate extends Aggregate {
   //~ Methods ----------------------------------------------------------------
 
   @Override public LogicalAggregate copy(RelTraitSet traitSet, RelNode input,
-      ImmutableBitSet groupSet, List<AggregateCall> aggCalls) {
+      boolean indicator, ImmutableBitSet groupSet,
+      List<ImmutableBitSet> groupSets, List<AggregateCall> aggCalls) {
     assert traitSet.containsIfApplicable(Convention.NONE);
-    return new LogicalAggregate(getCluster(), input, groupSet, aggCalls);
+    return new LogicalAggregate(getCluster(), input, this.indicator, groupSet,
+        groupSets, aggCalls);
   }
 
   @Override public RelNode accept(RelShuttle shuttle) {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6d79b5eb/core/src/main/java/org/apache/calcite/rel/rules/AggregateExpandDistinctAggregatesRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AggregateExpandDistinctAggregatesRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AggregateExpandDistinctAggregatesRule.java
index f513cad..4c017e0 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/AggregateExpandDistinctAggregatesRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/AggregateExpandDistinctAggregatesRule.java
@@ -77,7 +77,7 @@ public final class AggregateExpandDistinctAggregatesRule extends RelOptRule {
   public AggregateExpandDistinctAggregatesRule(
       Class<? extends LogicalAggregate> clazz,
       RelFactories.JoinFactory joinFactory) {
-    super(operand(clazz, any()));
+    super(operand(clazz, null, Aggregate.IS_SIMPLE, any()));
     this.joinFactory = joinFactory;
   }
 
@@ -154,11 +154,8 @@ public final class AggregateExpandDistinctAggregatesRule extends RelOptRule {
       rel = null;
     } else {
       rel =
-          new LogicalAggregate(
-              aggregate.getCluster(),
-              aggregate.getInput(),
-              groupSet,
-              newAggCallList);
+          new LogicalAggregate(aggregate.getCluster(), aggregate.getInput(),
+              false, groupSet, null, newAggCallList);
     }
 
     // For each set of operands, find and rewrite all calls which have that
@@ -201,12 +198,11 @@ public final class AggregateExpandDistinctAggregatesRule extends RelOptRule {
 
     // Create an aggregate on top, with the new aggregate list.
     final List<AggregateCall> newAggCalls =
-        new ArrayList<AggregateCall>(aggregate.getAggCallList());
+        Lists.newArrayList(aggregate.getAggCallList());
     rewriteAggCalls(newAggCalls, argList, sourceOf);
-    return aggregate.copy(
-        aggregate.getTraitSet(),
-        distinct,
-        ImmutableBitSet.range(aggregate.getGroupSet().cardinality()),
+    final int cardinality = aggregate.getGroupSet().cardinality();
+    return aggregate.copy(aggregate.getTraitSet(), distinct,
+        aggregate.indicator, ImmutableBitSet.range(cardinality), null,
         newAggCalls);
   }
 
@@ -342,11 +338,9 @@ public final class AggregateExpandDistinctAggregatesRule extends RelOptRule {
     }
 
     Aggregate distinctAgg =
-        aggregate.copy(
-            aggregate.getTraitSet(),
-            distinct,
+        aggregate.copy(aggregate.getTraitSet(), distinct, false,
             ImmutableBitSet.range(aggregate.getGroupSet().cardinality()),
-            aggCallList);
+            null, aggCallList);
 
     // If there's no left child yet, no need to create the join
     if (left == null) {
@@ -479,11 +473,9 @@ public final class AggregateExpandDistinctAggregatesRule extends RelOptRule {
 
     // Get the distinct values of the GROUP BY fields and the arguments
     // to the agg functions.
-    return aggregate.copy(
-        aggregate.getTraitSet(),
-        project,
+    return aggregate.copy(aggregate.getTraitSet(), project, false,
         ImmutableBitSet.range(projects.size()),
-        ImmutableList.<AggregateCall>of());
+        null, ImmutableList.<AggregateCall>of());
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6d79b5eb/core/src/main/java/org/apache/calcite/rel/rules/AggregateFilterTransposeRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AggregateFilterTransposeRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AggregateFilterTransposeRule.java
index 665191d..82c30f1 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/AggregateFilterTransposeRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/AggregateFilterTransposeRule.java
@@ -58,7 +58,7 @@ public class AggregateFilterTransposeRule extends RelOptRule {
 
   private AggregateFilterTransposeRule() {
     super(
-        operand(Aggregate.class,
+        operand(Aggregate.class, null, Aggregate.IS_SIMPLE,
             operand(Filter.class, any())));
   }
 
@@ -81,7 +81,7 @@ public class AggregateFilterTransposeRule extends RelOptRule {
       return;
     }
     final Aggregate newAggregate =
-        aggregate.copy(aggregate.getTraitSet(), input, newGroupSet,
+        aggregate.copy(aggregate.getTraitSet(), input, false, newGroupSet, null,
             aggregate.getAggCallList());
     final Mappings.TargetMapping mapping = Mappings.target(
         new Function<Integer, Integer>() {
@@ -124,8 +124,8 @@ public class AggregateFilterTransposeRule extends RelOptRule {
                 ImmutableList.of(i++), aggregateCall.type, aggregateCall.name));
       }
       final Aggregate topAggregate =
-          aggregate.copy(aggregate.getTraitSet(), newFilter,
-              topGroupSet.build(), topAggCallList);
+          aggregate.copy(aggregate.getTraitSet(), newFilter, false,
+              topGroupSet.build(), null, topAggCallList);
       call.transformTo(topAggregate);
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6d79b5eb/core/src/main/java/org/apache/calcite/rel/rules/AggregateProjectMergeRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AggregateProjectMergeRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AggregateProjectMergeRule.java
index e321d83..ebb882a 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/AggregateProjectMergeRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/AggregateProjectMergeRule.java
@@ -51,7 +51,7 @@ public class AggregateProjectMergeRule extends RelOptRule {
   /** Private constructor. */
   private AggregateProjectMergeRule() {
     super(
-        operand(Aggregate.class,
+        operand(Aggregate.class, null, Aggregate.IS_SIMPLE,
             operand(Project.class, any())));
   }
 
@@ -95,8 +95,8 @@ public class AggregateProjectMergeRule extends RelOptRule {
 
     final ImmutableBitSet newGroupSet = ImmutableBitSet.of(newKeys);
     final Aggregate newAggregate =
-        aggregate.copy(aggregate.getTraitSet(), project.getInput(), newGroupSet,
-            aggCalls.build());
+        aggregate.copy(aggregate.getTraitSet(), project.getInput(), false,
+            newGroupSet, null, aggCalls.build());
 
     // Add a project if the group set is not in the same order or
     // contains duplicates.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6d79b5eb/core/src/main/java/org/apache/calcite/rel/rules/AggregateProjectPullUpConstantsRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AggregateProjectPullUpConstantsRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AggregateProjectPullUpConstantsRule.java
index d696407..9ad2f2f 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/AggregateProjectPullUpConstantsRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/AggregateProjectPullUpConstantsRule.java
@@ -20,6 +20,7 @@ import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Aggregate;
 import org.apache.calcite.rel.core.AggregateCall;
 import org.apache.calcite.rel.logical.LogicalAggregate;
 import org.apache.calcite.rel.logical.LogicalProject;
@@ -72,7 +73,7 @@ public class AggregateProjectPullUpConstantsRule extends RelOptRule {
    */
   private AggregateProjectPullUpConstantsRule() {
     super(
-        operand(LogicalAggregate.class,
+        operand(LogicalAggregate.class, null, Aggregate.IS_SIMPLE,
             operand(LogicalProject.class, any())));
   }
 
@@ -137,11 +138,8 @@ public class AggregateProjectPullUpConstantsRule extends RelOptRule {
                 newGroupCount));
       }
       newAggregate =
-          new LogicalAggregate(
-              aggregate.getCluster(),
-              child,
-              ImmutableBitSet.range(newGroupCount),
-              newAggCalls);
+          new LogicalAggregate(aggregate.getCluster(), child, false,
+              ImmutableBitSet.range(newGroupCount), null, newAggCalls);
     } else {
       // Create the mapping from old field positions to new field
       // positions.
@@ -182,11 +180,8 @@ public class AggregateProjectPullUpConstantsRule extends RelOptRule {
 
       // Aggregate on projection.
       newAggregate =
-          new LogicalAggregate(
-              aggregate.getCluster(),
-              project,
-              ImmutableBitSet.range(newGroupCount),
-              newAggCalls);
+          new LogicalAggregate(aggregate.getCluster(), project, false,
+              ImmutableBitSet.range(newGroupCount), null, newAggCalls);
     }
 
     final RexBuilder rexBuilder = aggregate.getCluster().getRexBuilder();

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6d79b5eb/core/src/main/java/org/apache/calcite/rel/rules/AggregateReduceFunctionsRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AggregateReduceFunctionsRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AggregateReduceFunctionsRule.java
index 718063f..acdc65d 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/AggregateReduceFunctionsRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/AggregateReduceFunctionsRule.java
@@ -558,10 +558,8 @@ public class AggregateReduceFunctionsRule extends RelOptRule {
       Aggregate oldAggRel,
       RelNode inputRel,
       List<AggregateCall> newCalls) {
-    return new LogicalAggregate(
-        oldAggRel.getCluster(),
-        inputRel,
-        oldAggRel.getGroupSet(),
+    return new LogicalAggregate(oldAggRel.getCluster(), inputRel,
+        oldAggRel.indicator, oldAggRel.getGroupSet(), oldAggRel.getGroupSets(),
         newCalls);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6d79b5eb/core/src/main/java/org/apache/calcite/rel/rules/AggregateStarTableRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AggregateStarTableRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AggregateStarTableRule.java
index 019a01f..fd66ddf 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/AggregateStarTableRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/AggregateStarTableRule.java
@@ -56,13 +56,13 @@ import java.util.List;
 public class AggregateStarTableRule extends RelOptRule {
   public static final AggregateStarTableRule INSTANCE =
       new AggregateStarTableRule(
-          operand(Aggregate.class,
+          operand(Aggregate.class, null, Aggregate.IS_SIMPLE,
               some(operand(StarTable.StarTableScan.class, none()))),
           "AggregateStarTableRule");
 
   public static final AggregateStarTableRule INSTANCE2 =
       new AggregateStarTableRule(
-          operand(Aggregate.class,
+          operand(Aggregate.class, null, Aggregate.IS_SIMPLE,
               operand(Project.class,
                   operand(StarTable.StarTableScan.class, none()))),
           "AggregateStarTableRule:project") {
@@ -149,8 +149,8 @@ public class AggregateStarTableRule extends RelOptRule {
         }
         aggCalls.add(copy);
       }
-      rel = aggregate.copy(aggregate.getTraitSet(), rel, groupSet.build(),
-          aggCalls);
+      rel = aggregate.copy(aggregate.getTraitSet(), rel, false,
+          groupSet.build(), null, aggCalls);
     } else if (!tileKey.measures.equals(measures)) {
       System.out.println("Using materialization "
           + aggregateRelOptTable.getQualifiedName()

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6d79b5eb/core/src/main/java/org/apache/calcite/rel/rules/AggregateUnionAggregateRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AggregateUnionAggregateRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AggregateUnionAggregateRule.java
index f20cd69..1005f8a 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/AggregateUnionAggregateRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/AggregateUnionAggregateRule.java
@@ -19,6 +19,7 @@ package org.apache.calcite.rel.rules;
 import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Aggregate;
 import org.apache.calcite.rel.logical.LogicalAggregate;
 import org.apache.calcite.rel.logical.LogicalUnion;
 
@@ -48,7 +49,7 @@ public class AggregateUnionAggregateRule extends RelOptRule {
    */
   private AggregateUnionAggregateRule() {
     super(
-        operand(LogicalAggregate.class,
+        operand(LogicalAggregate.class, null, Aggregate.IS_SIMPLE,
             operand(LogicalUnion.class,
                 operand(RelNode.class, any()),
                 operand(RelNode.class, any()))));
@@ -101,11 +102,8 @@ public class AggregateUnionAggregateRule extends RelOptRule {
             true);
 
     LogicalAggregate newAggRel =
-        new LogicalAggregate(
-            topAggRel.getCluster(),
-            newUnion,
-            topAggRel.getGroupSet(),
-            topAggRel.getAggCallList());
+        new LogicalAggregate(topAggRel.getCluster(), newUnion, false,
+            topAggRel.getGroupSet(), null, topAggRel.getAggCallList());
 
     call.transformTo(newAggRel);
   }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6d79b5eb/core/src/main/java/org/apache/calcite/rel/rules/AggregateUnionTransposeRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AggregateUnionTransposeRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AggregateUnionTransposeRule.java
index 7b92704..3aae4ff 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/AggregateUnionTransposeRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/AggregateUnionTransposeRule.java
@@ -22,6 +22,7 @@ import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Aggregate;
 import org.apache.calcite.rel.core.AggregateCall;
 import org.apache.calcite.rel.logical.LogicalAggregate;
 import org.apache.calcite.rel.logical.LogicalUnion;
@@ -66,7 +67,7 @@ public class AggregateUnionTransposeRule extends RelOptRule {
    */
   private AggregateUnionTransposeRule() {
     super(
-        operand(LogicalAggregate.class,
+        operand(LogicalAggregate.class, null, Aggregate.IS_SIMPLE,
             operand(LogicalUnion.class, any())));
   }
 
@@ -114,10 +115,8 @@ public class AggregateUnionTransposeRule extends RelOptRule {
       } else {
         anyTransformed = true;
         newUnionInputs.add(
-            new LogicalAggregate(
-                cluster, input,
-                aggRel.getGroupSet(),
-                aggRel.getAggCallList()));
+            new LogicalAggregate(cluster, input, false, aggRel.getGroupSet(),
+                null, aggRel.getAggCallList()));
       }
     }
 
@@ -131,11 +130,9 @@ public class AggregateUnionTransposeRule extends RelOptRule {
     // create a new union whose children are the aggs created above
     LogicalUnion newUnion = new LogicalUnion(cluster, newUnionInputs, true);
 
-    LogicalAggregate newTopAggRel = new LogicalAggregate(
-        cluster,
-        newUnion,
-        aggRel.getGroupSet(),
-        transformedAggCalls);
+    LogicalAggregate newTopAggRel =
+        new LogicalAggregate(cluster, newUnion, false, aggRel.getGroupSet(),
+            null, transformedAggCalls);
 
     // In case we transformed any COUNT (which is always NOT NULL)
     // to SUM (which is always NULLABLE), cast back to keep the
@@ -168,7 +165,7 @@ public class AggregateUnionTransposeRule extends RelOptRule {
         // inputs nor we'll face empty set.
         aggType = null;
       } else {
-        aggFun = (SqlAggFunction) origCall.getAggregation();
+        aggFun = origCall.getAggregation();
         aggType = origCall.getType();
       }
       AggregateCall newCall =

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6d79b5eb/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 733a360..ceb7b49 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlCallBinding.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlCallBinding.java
@@ -74,8 +74,16 @@ public class SqlCallBinding extends SqlOperatorBinding {
       return 0;
     }
     final SqlSelect select = selectScope.getNode();
-    if (select.getGroup() != null) {
-      return select.getGroup().size();
+    final SqlNodeList group = select.getGroup();
+    if (group != null) {
+      int n = 0;
+      for (SqlNode groupItem : group) {
+        if (!(groupItem instanceof SqlNodeList)
+            || ((SqlNodeList) groupItem).size() != 0) {
+          ++n;
+        }
+      }
+      return n;
     }
     return validator.isAggregate(select) ? 0 : -1;
   }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6d79b5eb/core/src/main/java/org/apache/calcite/sql/SqlInternalOperator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlInternalOperator.java b/core/src/main/java/org/apache/calcite/sql/SqlInternalOperator.java
index 4eebe26..deba96b 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlInternalOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlInternalOperator.java
@@ -16,27 +16,42 @@
  */
 package org.apache.calcite.sql;
 
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.ReturnTypes;
 import org.apache.calcite.sql.type.SqlOperandTypeChecker;
 import org.apache.calcite.sql.type.SqlOperandTypeInference;
 import org.apache.calcite.sql.type.SqlReturnTypeInference;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql.validate.SqlValidatorScope;
 
 /**
  * Generic operator for nodes with internal syntax.
+ *
+ * <p>If you do not override {@link #getSyntax()} or
+ * {@link #unparse(SqlWriter, SqlCall, int, int)}, they will be unparsed using
+ * function syntax, {@code F(arg1, arg2, ...)}. This may be OK for operators
+ * that never appear in SQL, only as structural elements in an abstract syntax
+ * tree.
+ *
+ * <p>You can use this operator, without creating a sub-class, for
+ * non-expression nodes. Validate will validate the arguments, but will not
+ * attempt to deduce a type.
  */
-public abstract class SqlInternalOperator extends SqlSpecialOperator {
+public class SqlInternalOperator extends SqlSpecialOperator {
   //~ Constructors -----------------------------------------------------------
 
   public SqlInternalOperator(
       String name,
       SqlKind kind) {
-    super(name, kind, 2, true, null, null, null);
+    super(name, kind, 2, true, ReturnTypes.ARG0, null, OperandTypes.VARIADIC);
   }
 
   public SqlInternalOperator(
       String name,
       SqlKind kind,
       int prec) {
-    super(name, kind, prec, true, null, null, null);
+    super(name, kind, prec, true, ReturnTypes.ARG0, null, null);
   }
 
   public SqlInternalOperator(
@@ -60,16 +75,12 @@ public abstract class SqlInternalOperator extends SqlSpecialOperator {
   //~ Methods ----------------------------------------------------------------
 
   public SqlSyntax getSyntax() {
-    return SqlSyntax.INTERNAL;
+    return SqlSyntax.FUNCTION;
   }
 
-  public void unparse(
-      SqlWriter writer,
-      SqlCall call,
-      int leftPrec,
-      int rightPrec) {
-    throw new UnsupportedOperationException(
-        "unparse must be implemented by SqlCall subclass");
+  @Override public RelDataType deriveType(SqlValidator validator,
+      SqlValidatorScope scope, SqlCall call) {
+    return validateOperands(validator, scope, call);
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6d79b5eb/core/src/main/java/org/apache/calcite/sql/SqlKind.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlKind.java b/core/src/main/java/org/apache/calcite/sql/SqlKind.java
index a6da09d..6c6fd85 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlKind.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlKind.java
@@ -563,7 +563,19 @@ public enum SqlKind {
    * The internal REINTERPRET operator (meaning a reinterpret cast).
    * An internal operator that does not appear in SQL syntax.
    */
-  REINTERPRET;
+  REINTERPRET,
+
+  /** The internal {@code CUBE} operator that occurs within a {@code GROUP BY}
+   * clause. */
+  CUBE,
+
+  /** The internal {@code ROLLUP} operator that occurs within a {@code GROUP BY}
+   * clause. */
+  ROLLUP,
+
+  /** The internal {@code GROUPING SETS} operator that occurs within a
+   * {@code GROUP BY} clause. */
+  GROUPING_SETS;
 
   //~ Static fields/initializers ---------------------------------------------
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6d79b5eb/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 9425514..cab1ecc 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlOperator.java
@@ -65,7 +65,7 @@ public abstract class SqlOperator {
   /**
    * Maximum precedence.
    */
-  protected static final int MDX_PRECEDENCE = 200;
+  public static final int MDX_PRECEDENCE = 200;
 
   //~ Instance fields --------------------------------------------------------
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6d79b5eb/core/src/main/java/org/apache/calcite/sql/SqlSpecialOperator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlSpecialOperator.java b/core/src/main/java/org/apache/calcite/sql/SqlSpecialOperator.java
index efecf9d..22b748d 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlSpecialOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlSpecialOperator.java
@@ -66,15 +66,6 @@ public class SqlSpecialOperator extends SqlOperator {
     return SqlSyntax.SPECIAL;
   }
 
-  public void unparse(
-      SqlWriter writer,
-      SqlCall call,
-      int leftPrec,
-      int rightPrec) {
-    throw new UnsupportedOperationException(
-        "unparse must be implemented by SqlCall subclass");
-  }
-
   /**
    * Reduces a list of operators and arguments according to the rules of
    * precedence and associativity. Returns the ordinal of the node which

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6d79b5eb/core/src/main/java/org/apache/calcite/sql/fun/SqlLiteralChainOperator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlLiteralChainOperator.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlLiteralChainOperator.java
index d51ef0c..a3d6f24 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlLiteralChainOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlLiteralChainOperator.java
@@ -22,11 +22,11 @@ import org.apache.calcite.sql.SqlCall;
 import org.apache.calcite.sql.SqlCallBinding;
 import org.apache.calcite.sql.SqlCharStringLiteral;
 import org.apache.calcite.sql.SqlCollation;
-import org.apache.calcite.sql.SqlInternalOperator;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlLiteral;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlOperatorBinding;
+import org.apache.calcite.sql.SqlSpecialOperator;
 import org.apache.calcite.sql.SqlUtil;
 import org.apache.calcite.sql.SqlWriter;
 import org.apache.calcite.sql.parser.SqlParserPos;
@@ -56,7 +56,7 @@ import static org.apache.calcite.util.Static.RESOURCE;
  * objects, a {@link SqlCollation} object is attached only to the head of the
  * chain.
  */
-public class SqlLiteralChainOperator extends SqlInternalOperator {
+public class SqlLiteralChainOperator extends SqlSpecialOperator {
   //~ Constructors -----------------------------------------------------------
 
   SqlLiteralChainOperator() {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6d79b5eb/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java
index 465fcc9..905b7ae 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java
@@ -148,6 +148,21 @@ public class SqlStdOperatorTable extends ReflectiveSqlOperatorTable {
    */
   public static final SqlAsOperator AS = new SqlAsOperator();
 
+  /** {@code CUBE} operator, occurs within {@code GROUP BY} clause
+   * or nested within a {@code GROUPING SETS}. */
+  public static final SqlInternalOperator CUBE =
+      new SqlInternalOperator("CUBE", SqlKind.CUBE);
+
+  /** {@code ROLLUP} operator, occurs within {@code GROUP BY} clause
+   * or nested within a {@code GROUPING SETS}. */
+  public static final SqlInternalOperator ROLLUP =
+      new SqlInternalOperator("ROLLUP", SqlKind.ROLLUP);
+
+  /** {@code GROUPING SETS} operator, occurs within {@code GROUP BY} clause
+   * or nested within a {@code GROUPING SETS}. */
+  public static final SqlInternalOperator GROUPING_SETS =
+      new SqlInternalOperator("GROUPING_SETS", SqlKind.GROUPING_SETS);
+
   /**
    * String concatenation operator, '<code>||</code>'.
    */
@@ -937,7 +952,7 @@ public class SqlStdOperatorTable extends ReflectiveSqlOperatorTable {
   public static final SqlLiteralChainOperator LITERAL_CHAIN =
       new SqlLiteralChainOperator();
 
-  public static final SqlInternalOperator THROW = new SqlThrowOperator();
+  public static final SqlThrowOperator THROW = new SqlThrowOperator();
 
   public static final SqlBetweenOperator BETWEEN =
       new SqlBetweenOperator(

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6d79b5eb/core/src/main/java/org/apache/calcite/sql/fun/SqlThrowOperator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlThrowOperator.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlThrowOperator.java
index f223141..fdb380e 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlThrowOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlThrowOperator.java
@@ -17,8 +17,8 @@
 package org.apache.calcite.sql.fun;
 
 import org.apache.calcite.sql.SqlCall;
-import org.apache.calcite.sql.SqlInternalOperator;
 import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlSpecialOperator;
 import org.apache.calcite.sql.SqlWriter;
 import org.apache.calcite.sql.type.OperandTypes;
 import org.apache.calcite.sql.type.ReturnTypes;
@@ -37,7 +37,7 @@ import org.apache.calcite.sql.type.ReturnTypes;
  * ELSE throw("what's wrong with you man?")<br>
  * END</code></blockquote>
  */
-public class SqlThrowOperator extends SqlInternalOperator {
+public class SqlThrowOperator extends SqlSpecialOperator {
   //~ Constructors -----------------------------------------------------------
 
   public SqlThrowOperator() {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6d79b5eb/core/src/main/java/org/apache/calcite/sql/validate/AggChecker.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/AggChecker.java b/core/src/main/java/org/apache/calcite/sql/validate/AggChecker.java
index 1b7d3a8..3628d63 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/AggChecker.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/AggChecker.java
@@ -22,9 +22,12 @@ import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlUtil;
 import org.apache.calcite.sql.util.SqlBasicVisitor;
+import org.apache.calcite.util.Stacks;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
 
 import java.util.List;
-import java.util.Stack;
 
 import static org.apache.calcite.util.Static.RESOURCE;
 
@@ -35,9 +38,8 @@ import static org.apache.calcite.util.Static.RESOURCE;
 class AggChecker extends SqlBasicVisitor<Void> {
   //~ Instance fields --------------------------------------------------------
 
-  private final Stack<SqlValidatorScope> scopes =
-      new Stack<SqlValidatorScope>();
-  private final List<SqlNode> groupExprs;
+  private final List<SqlValidatorScope> scopes = Lists.newArrayList();
+  private final ImmutableList<SqlNode> groupExprs;
   private boolean distinct;
   private SqlValidatorImpl validator;
 
@@ -59,9 +61,9 @@ class AggChecker extends SqlBasicVisitor<Void> {
       List<SqlNode> groupExprs,
       boolean distinct) {
     this.validator = validator;
-    this.groupExprs = groupExprs;
+    this.groupExprs = ImmutableList.copyOf(groupExprs);
     this.distinct = distinct;
-    this.scopes.push(scope);
+    Stacks.push(this.scopes, scope);
   }
 
   //~ Methods ----------------------------------------------------------------
@@ -98,7 +100,7 @@ class AggChecker extends SqlBasicVisitor<Void> {
     // it fully-qualified.
     // TODO: It would be better if we always compared fully-qualified
     // to fully-qualified.
-    final SqlIdentifier fqId = scopes.peek().fullyQualify(id);
+    final SqlIdentifier fqId = Stacks.peek(scopes).fullyQualify(id);
     if (isGroupExpr(fqId)) {
       return null;
     }
@@ -135,16 +137,16 @@ class AggChecker extends SqlBasicVisitor<Void> {
     }
 
     // Switch to new scope.
-    SqlValidatorScope oldScope = scopes.peek();
+    SqlValidatorScope oldScope = Stacks.peek(scopes);
     SqlValidatorScope newScope = oldScope.getOperandScope(call);
-    scopes.push(newScope);
+    Stacks.push(scopes, newScope);
 
     // Visit the operands (only expressions).
     call.getOperator()
         .acceptCall(this, call, true, ArgHandlerImpl.<Void>instance());
 
     // Restore scope.
-    scopes.pop();
+    Stacks.pop(scopes, newScope);
     return null;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6d79b5eb/core/src/main/java/org/apache/calcite/sql/validate/AggregatingSelectScope.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/AggregatingSelectScope.java b/core/src/main/java/org/apache/calcite/sql/validate/AggregatingSelectScope.java
index 27875ba..b6d6c8d 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/AggregatingSelectScope.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/AggregatingSelectScope.java
@@ -21,6 +21,8 @@ import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlNodeList;
 import org.apache.calcite.sql.SqlSelect;
 
+import com.google.common.collect.Lists;
+
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
@@ -70,7 +72,10 @@ public class AggregatingSelectScope
       SqlNodeList sqlNodeList =
           (SqlNodeList) this.select.getGroup().accept(
               new SqlValidatorUtil.DeepCopier(parent));
-      this.groupExprList = sqlNodeList.getList();
+      groupExprList = Lists.newArrayList();
+      for (SqlNode node : sqlNodeList) {
+        addGroupExpr(node);
+      }
     } else {
       groupExprList = null;
     }
@@ -179,7 +184,23 @@ public class AggregatingSelectScope
    * @param expr Expression
    */
   public void addGroupExpr(SqlNode expr) {
-    groupExprList.add(expr);
+    switch (expr.getKind()) {
+    case CUBE:
+    case GROUPING_SETS:
+    case ROLLUP:
+    case ROW:
+      for (SqlNode child : ((SqlCall) expr).getOperandList()) {
+        addGroupExpr(child);
+      }
+      break;
+    default:
+      for (SqlNode existingNode : groupExprList) {
+        if (existingNode.equalsDeep(expr, false)) {
+          return;
+        }
+      }
+      groupExprList.add(expr);
+    }
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6d79b5eb/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 5dcf320..c3ff914 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
@@ -3035,16 +3035,42 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
       aggregatingScope = (AggregatingSelectScope) selectScope;
     }
     for (SqlNode groupItem : groupList) {
+      if (groupItem instanceof SqlNodeList
+          && ((SqlNodeList) groupItem).size() == 0) {
+        continue;
+      }
+      validateGroupItem(groupScope, aggregatingScope, groupItem);
+    }
+
+    SqlNode agg = aggFinder.findAgg(groupList);
+    if (agg != null) {
+      throw newValidationError(agg, RESOURCE.aggregateIllegalInGroupBy());
+    }
+  }
+
+  private void validateGroupItem(SqlValidatorScope groupScope,
+      AggregatingSelectScope aggregatingScope,
+      SqlNode groupItem) {
+    switch (groupItem.getKind()) {
+    case GROUPING_SETS:
+      validateGroupingSets(groupScope, aggregatingScope, (SqlCall) groupItem);
+      break;
+    default:
+      if (groupItem instanceof SqlNodeList) {
+        break;
+      }
       final RelDataType type = deriveType(groupScope, groupItem);
       setValidatedNodeTypeImpl(groupItem, type);
       if (aggregatingScope != null) {
         aggregatingScope.addGroupExpr(groupItem);
       }
     }
+  }
 
-    SqlNode agg = aggFinder.findAgg(groupList);
-    if (agg != null) {
-      throw newValidationError(agg, RESOURCE.aggregateIllegalInGroupBy());
+  private void validateGroupingSets(SqlValidatorScope groupScope,
+      AggregatingSelectScope aggregatingScope, SqlCall groupItem) {
+    for (SqlNode node : groupItem.getOperandList()) {
+      validateGroupItem(groupScope, aggregatingScope, node);
     }
   }
 


[39/58] [abbrv] [partial] incubator-calcite git commit: [CALCITE-306] Standardize code style for "import package.*; "

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/plan/volcano/package-info.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/volcano/package-info.java b/core/src/main/java/org/apache/calcite/plan/volcano/package-info.java
index 0e6a0fc..482a3ee 100644
--- a/core/src/main/java/org/apache/calcite/plan/volcano/package-info.java
+++ b/core/src/main/java/org/apache/calcite/plan/volcano/package-info.java
@@ -22,18 +22,18 @@
  *
  * <p>A <dfn>planner</dfn> (also known as an <dfn>optimizer</dfn>) finds the
  * most efficient implementation of a
- * {@link org.eigenbase.rel.RelNode relational expression}.</p>
+ * {@link org.apache.calcite.rel.RelNode relational expression}.</p>
  *
- * <p>Interface {@link org.eigenbase.relopt.RelOptPlanner} defines a planner,
- * and class {@link org.eigenbase.relopt.volcano.VolcanoPlanner} is an
+ * <p>Interface {@link org.apache.calcite.plan.RelOptPlanner} defines a planner,
+ * and class {@link org.apache.calcite.plan.volcano.VolcanoPlanner} is an
  * implementation which uses a dynamic programming technique. It is based upon
  * the Volcano optimizer [<a href="#graefe93">1</a>].</p>
  *
- * <p>Interface {@link org.eigenbase.relopt.RelOptCost} defines a cost
- * model; class {@link org.eigenbase.relopt.volcano.VolcanoCost} is
+ * <p>Interface {@link org.apache.calcite.plan.RelOptCost} defines a cost
+ * model; class {@link org.apache.calcite.plan.volcano.VolcanoCost} is
  * the implementation for a <code>VolcanoPlanner</code>.</p>
  *
- * <p>A {@link org.eigenbase.relopt.volcano.RelSet} is a set of equivalent
+ * <p>A {@link org.apache.calcite.plan.volcano.RelSet} is a set of equivalent
  * relational expressions.  They are equivalent because they will produce the
  * same result for any set of input data. It is an equivalence class: two
  * expressions are in the same set if and only if they are in the same
@@ -42,25 +42,25 @@
  * <p>One of the unique features of the optimizer is that expressions can take
  * on a variety of physical traits. Each relational expression has a set of
  * traits. Each trait is described by an implementation of
- * {@link org.eigenbase.relopt.RelTraitDef}.  Manifestations of the trait
- * implement {@link org.eigenbase.relopt.RelTrait}. The most common example of a
- * trait is calling convention: the protocol used to receive and transmit
- * data. {@link org.eigenbase.relopt.ConventionTraitDef} defines the trait and
- * {@link org.eigenbase.relopt.Convention} enumerates the protocols. Every
- * relational expression has a single calling convention by which it returns its
- * results. Some examples:</p>
+ * {@link org.apache.calcite.plan.RelTraitDef}.  Manifestations of the trait
+ * implement {@link org.apache.calcite.plan.RelTrait}. The most common example
+ * of a trait is calling convention: the protocol used to receive and transmit
+ * data. {@link org.apache.calcite.plan.ConventionTraitDef} defines the trait
+ * and {@link org.apache.calcite.plan.Convention} enumerates the
+ * protocols. Every relational expression has a single calling convention by
+ * which it returns its results. Some examples:</p>
  *
  * <ul>
- *     <li>{@link net.hydromatic.optiq.impl.jdbc.JdbcConvention} is a fairly
+ *     <li>{@link org.apache.calcite.adapter.jdbc.JdbcConvention} is a fairly
  *         conventional convention; the results are rows from a
  *         {@link java.sql.ResultSet JDBC result set}.
  *     </li>
- *     <li>{@link org.eigenbase.relopt.Convention#NONE} means that a
+ *     <li>{@link org.apache.calcite.plan.Convention#NONE} means that a
  *         relational
  *         expression cannot be implemented; typically there are rules which can
  *         transform it to equivalent, implementable expressions.
  *     </li>
- *     <li>{@link net.hydromatic.optiq.rules.java.EnumerableConvention}
+ *     <li>{@link org.apache.calcite.adapter.enumerable.EnumerableConvention}
  *         implements the expression by
  *         generating Java code. The code places the current row in a Java
  *         variable, then
@@ -80,41 +80,40 @@
  *
  * <p>New traits are added to the planner in one of two ways:</p>
  * <ol>
- *     <li>If the new trait is integral to Calcite, then each and every
- *         implementation of {@link org.eigenbase.rel.RelNode} should include
- *         its manifestation of the trait as part of the
- *         {@link org.eigenbase.relopt.RelTraitSet} passed to
- *         {@link org.eigenbase.rel.AbstractRelNode}'s constructor. It may be
- *         useful to provide alternate <code>AbstractRelNode</code> constructors
- *         if most relational expressions use a single manifestation of the
- *         trait.</li>
- *
- *     <li>If the new trait describes some aspect of a Farrago extension, then
- *         the RelNodes passed to
- *         {@link org.eigenbase.relopt.volcano.VolcanoPlanner#setRoot(org.eigenbase.rel.RelNode)}
- *         should have their trait sets expanded before the
- *         <code>setRoot(RelNode)</code> call.</li>
+ * <li>If the new trait is integral to Calcite, then each and every
+ *     implementation of {@link org.apache.calcite.rel.RelNode} should include
+ *     its manifestation of the trait as part of the
+ *     {@link org.apache.calcite.plan.RelTraitSet} passed to
+ *     {@link org.apache.calcite.rel.AbstractRelNode}'s constructor. It may be
+ *     useful to provide alternate <code>AbstractRelNode</code> constructors
+ *     if most relational expressions use a single manifestation of the
+ *     trait.</li>
+ *
+ * <li>If the new trait describes some aspect of a Farrago extension, then
+ *     the RelNodes passed to
+ *     {@link org.apache.calcite.plan.volcano.VolcanoPlanner#setRoot(org.apache.calcite.rel.RelNode)}
+ *     should have their trait sets expanded before the
+ *     <code>setRoot(RelNode)</code> call.</li>
  *
  * </ol>
  *
- * <p>The second trait extension mechanism requires that implementations of {@link
- *     org.eigenbase.rel.AbstractRelNode#clone()} must not assume the type and
- *     quantity of traits in
- *     their trait set. In either case, the new <code>RelTraitDef</code>
- *     implementation must be
- *     {@link org.eigenbase.relopt.volcano.VolcanoPlanner#addRelTraitDef(org.eigenbase.relopt.RelTraitDef)}
- *     registered with the planner.</p>
+ * <p>The second trait extension mechanism requires that implementations of
+ * {@link org.apache.calcite.rel.AbstractRelNode#clone()} must not assume the
+ * type and quantity of traits in their trait set. In either case, the new
+ * <code>RelTraitDef</code> implementation must be
+ * {@link org.apache.calcite.plan.volcano.VolcanoPlanner#addRelTraitDef(org.apache.calcite.plan.RelTraitDef)}
+ * registered with the planner.</p>
  *
- * <p>A {@link org.eigenbase.relopt.volcano.RelSubset} is a subset of a <code>RelSet</code>
- *     containing expressions which are equivalent and which have the same
- *     <code>Convention</code>. Like <code>RelSet</code>, it is an equivalence
- *     class.</p>
+ * <p>A {@link org.apache.calcite.plan.volcano.RelSubset} is a subset of a
+ * <code>RelSet</code> containing expressions which are equivalent and which
+ * have the same <code>Convention</code>. Like <code>RelSet</code>, it is an
+ * equivalence class.</p>
  *
  * <h2>Related packages</h2>
  * <ul>
- *     <li>{@code <a href="../rel/package-summary.html">org.eigenbase.rel</a>}
- *         defines {@link org.eigenbase.rel.RelNode relational expressions}.
- *     </li>
+ * <li>{@code <a href="../rel/package-summary.html">org.apache.calcite.rel</a>}
+ *     defines {@link org.apache.calcite.rel.RelNode relational expressions}.
+ * </li>
  * </ul>
  *
  * <h2>Details</h2>
@@ -269,6 +268,6 @@
  *     McKenna
  *     (1993)</a>.</p>
  */
-package org.eigenbase.relopt.volcano;
+package org.apache.calcite.plan.volcano;
 
 // End package-info.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/prepare/CalciteCatalogReader.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/prepare/CalciteCatalogReader.java b/core/src/main/java/org/apache/calcite/prepare/CalciteCatalogReader.java
index a000b3a..0f4b4f5 100644
--- a/core/src/main/java/org/apache/calcite/prepare/CalciteCatalogReader.java
+++ b/core/src/main/java/org/apache/calcite/prepare/CalciteCatalogReader.java
@@ -14,40 +14,69 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.prepare;
+package org.apache.calcite.prepare;
 
-import net.hydromatic.optiq.*;
-import net.hydromatic.optiq.Function;
-import net.hydromatic.optiq.Table;
-import net.hydromatic.optiq.impl.java.JavaTypeFactory;
-import net.hydromatic.optiq.jdbc.OptiqSchema;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.jdbc.CalciteSchema;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeFactoryImpl;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.schema.AggregateFunction;
+import org.apache.calcite.schema.Function;
+import org.apache.calcite.schema.FunctionParameter;
+import org.apache.calcite.schema.ScalarFunction;
+import org.apache.calcite.schema.Schemas;
+import org.apache.calcite.schema.Table;
+import org.apache.calcite.schema.TableFunction;
+import org.apache.calcite.schema.TableMacro;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlOperatorTable;
+import org.apache.calcite.sql.SqlSyntax;
+import org.apache.calcite.sql.type.InferTypes;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.ReturnTypes;
+import org.apache.calcite.sql.type.SqlTypeFamily;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.validate.SqlMoniker;
+import org.apache.calcite.sql.validate.SqlMonikerImpl;
+import org.apache.calcite.sql.validate.SqlMonikerType;
+import org.apache.calcite.sql.validate.SqlUserDefinedAggFunction;
+import org.apache.calcite.sql.validate.SqlUserDefinedFunction;
+import org.apache.calcite.sql.validate.SqlUserDefinedTableFunction;
+import org.apache.calcite.sql.validate.SqlUserDefinedTableMacro;
+import org.apache.calcite.sql.validate.SqlValidatorUtil;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
 
-import org.eigenbase.relopt.RelOptPlanner;
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.sql.validate.*;
-import org.eigenbase.util.Pair;
-import org.eigenbase.util.Util;
+import com.google.common.collect.Collections2;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
 
-import com.google.common.collect.*;
-
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableSet;
 
 /**
- * Implementation of {@link net.hydromatic.optiq.prepare.Prepare.CatalogReader}
- * and also {@link org.eigenbase.sql.SqlOperatorTable} based on tables and
+ * Implementation of {@link org.apache.calcite.prepare.Prepare.CatalogReader}
+ * and also {@link org.apache.calcite.sql.SqlOperatorTable} based on tables and
  * functions defined schemas.
  */
-public class OptiqCatalogReader implements Prepare.CatalogReader,
+public class CalciteCatalogReader implements Prepare.CatalogReader,
     SqlOperatorTable {
-  final OptiqSchema rootSchema;
+  final CalciteSchema rootSchema;
   final JavaTypeFactory typeFactory;
   private final List<String> defaultSchema;
   private final boolean caseSensitive;
 
-  public OptiqCatalogReader(
-      OptiqSchema rootSchema,
+  public CalciteCatalogReader(
+      CalciteSchema rootSchema,
       boolean caseSensitive,
       List<String> defaultSchema,
       JavaTypeFactory typeFactory) {
@@ -59,8 +88,8 @@ public class OptiqCatalogReader implements Prepare.CatalogReader,
     this.typeFactory = typeFactory;
   }
 
-  public OptiqCatalogReader withSchemaPath(List<String> schemaPath) {
-    return new OptiqCatalogReader(rootSchema, caseSensitive, schemaPath,
+  public CalciteCatalogReader withSchemaPath(List<String> schemaPath) {
+    return new CalciteCatalogReader(rootSchema, caseSensitive, schemaPath,
         typeFactory);
   }
 
@@ -78,7 +107,7 @@ public class OptiqCatalogReader implements Prepare.CatalogReader,
 
   private RelOptTableImpl getTableFrom(List<String> names,
       List<String> schemaNames) {
-    OptiqSchema schema =
+    CalciteSchema schema =
         getSchema(Iterables.concat(schemaNames, Util.skipLast(names)));
     if (schema == null) {
       return null;
@@ -104,7 +133,7 @@ public class OptiqCatalogReader implements Prepare.CatalogReader,
       // If name is qualified, ignore path.
       schemaNameList = ImmutableList.of(ImmutableList.<String>of());
     } else {
-      OptiqSchema schema = getSchema(defaultSchema);
+      CalciteSchema schema = getSchema(defaultSchema);
       if (schema == null) {
         schemaNameList = ImmutableList.of();
       } else {
@@ -112,7 +141,7 @@ public class OptiqCatalogReader implements Prepare.CatalogReader,
       }
     }
     for (List<String> schemaNames : schemaNameList) {
-      OptiqSchema schema =
+      CalciteSchema schema =
           getSchema(Iterables.concat(schemaNames, Util.skipLast(names)));
       if (schema != null) {
         final String name = Util.last(names);
@@ -122,8 +151,8 @@ public class OptiqCatalogReader implements Prepare.CatalogReader,
     return functions2;
   }
 
-  private OptiqSchema getSchema(Iterable<String> schemaNames) {
-    OptiqSchema schema = rootSchema;
+  private CalciteSchema getSchema(Iterable<String> schemaNames) {
+    CalciteSchema schema = rootSchema;
     for (String schemaName : schemaNames) {
       schema = schema.getSubSchema(schemaName, caseSensitive);
       if (schema == null) {
@@ -138,12 +167,12 @@ public class OptiqCatalogReader implements Prepare.CatalogReader,
   }
 
   public List<SqlMoniker> getAllSchemaObjectNames(List<String> names) {
-    final OptiqSchema schema = getSchema(names);
+    final CalciteSchema schema = getSchema(names);
     if (schema == null) {
       return ImmutableList.of();
     }
     final List<SqlMoniker> result = new ArrayList<SqlMoniker>();
-    final Map<String, OptiqSchema> schemaMap = schema.getSubSchemaMap();
+    final Map<String, CalciteSchema> schemaMap = schema.getSubSchemaMap();
 
     for (String subSchema : schemaMap.keySet()) {
       result.add(
@@ -271,4 +300,4 @@ public class OptiqCatalogReader implements Prepare.CatalogReader,
   }
 }
 
-// End OptiqCatalogReader.java
+// End CalciteCatalogReader.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/prepare/CalciteMaterializer.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/prepare/CalciteMaterializer.java b/core/src/main/java/org/apache/calcite/prepare/CalciteMaterializer.java
index b089432..118214e 100644
--- a/core/src/main/java/org/apache/calcite/prepare/CalciteMaterializer.java
+++ b/core/src/main/java/org/apache/calcite/prepare/CalciteMaterializer.java
@@ -14,33 +14,49 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.prepare;
-
-import net.hydromatic.optiq.*;
-import net.hydromatic.optiq.impl.StarTable;
-import net.hydromatic.optiq.jdbc.OptiqPrepare;
-import net.hydromatic.optiq.jdbc.OptiqSchema;
-import net.hydromatic.optiq.rules.java.EnumerableConvention;
-import net.hydromatic.optiq.rules.java.EnumerableRel;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.sql.SqlNode;
-import org.eigenbase.sql.parser.SqlParseException;
-import org.eigenbase.sql.parser.SqlParser;
-import org.eigenbase.sql2rel.SqlToRelConverter;
+package org.apache.calcite.prepare;
+
+import org.apache.calcite.adapter.enumerable.EnumerableConvention;
+import org.apache.calcite.adapter.enumerable.EnumerableRel;
+import org.apache.calcite.jdbc.CalcitePrepare;
+import org.apache.calcite.jdbc.CalciteSchema;
+import org.apache.calcite.plan.RelOptMaterialization;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelShuttle;
+import org.apache.calcite.rel.core.Correlator;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.core.TableFunctionScan;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.rel.logical.LogicalAggregate;
+import org.apache.calcite.rel.logical.LogicalFilter;
+import org.apache.calcite.rel.logical.LogicalIntersect;
+import org.apache.calcite.rel.logical.LogicalJoin;
+import org.apache.calcite.rel.logical.LogicalMinus;
+import org.apache.calcite.rel.logical.LogicalProject;
+import org.apache.calcite.rel.logical.LogicalUnion;
+import org.apache.calcite.rel.logical.LogicalValues;
+import org.apache.calcite.schema.Schemas;
+import org.apache.calcite.schema.Table;
+import org.apache.calcite.schema.impl.StarTable;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.parser.SqlParseException;
+import org.apache.calcite.sql.parser.SqlParser;
+import org.apache.calcite.sql2rel.SqlToRelConverter;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 
-import java.util.*;
+import java.util.List;
 
 /**
  * Context for populating a {@link Materialization}.
  */
-class OptiqMaterializer extends OptiqPrepareImpl.OptiqPreparingStmt {
-  public OptiqMaterializer(OptiqPrepare.Context context,
-      CatalogReader catalogReader, OptiqSchema schema,
+class CalciteMaterializer extends CalcitePrepareImpl.CalcitePreparingStmt {
+  public CalciteMaterializer(CalcitePrepare.Context context,
+      CatalogReader catalogReader, CalciteSchema schema,
       RelOptPlanner planner) {
     super(context, catalogReader, catalogReader.getTypeFactory(), schema,
         EnumerableRel.Prefer.ANY, planner, EnumerableConvention.INSTANCE);
@@ -81,11 +97,11 @@ class OptiqMaterializer extends OptiqPrepareImpl.OptiqPreparingStmt {
   /** Converts a relational expression to use a
    * {@link StarTable} defined in {@code schema}.
    * Uses the first star table that fits. */
-  private void useStar(OptiqSchema schema, Materialization materialization) {
+  private void useStar(CalciteSchema schema, Materialization materialization) {
     for (Callback x : useStar(schema, materialization.queryRel)) {
       // Success -- we found a star table that matches.
       materialization.materialize(x.rel, x.starRelOptTable);
-      if (OptiqPrepareImpl.DEBUG) {
+      if (CalcitePrepareImpl.DEBUG) {
         System.out.println("Materialization "
             + materialization.materializedTable + " matched star table "
             + x.starTable + "; query after re-write: "
@@ -95,10 +111,10 @@ class OptiqMaterializer extends OptiqPrepareImpl.OptiqPreparingStmt {
   }
 
   /** Converts a relational expression to use a
-   * {@link net.hydromatic.optiq.impl.StarTable} defined in {@code schema}.
+   * {@link org.apache.calcite.schema.impl.StarTable} defined in {@code schema}.
    * Uses the first star table that fits. */
-  private Iterable<Callback> useStar(OptiqSchema schema, RelNode queryRel) {
-    List<OptiqSchema.TableEntry> starTables =
+  private Iterable<Callback> useStar(CalciteSchema schema, RelNode queryRel) {
+    List<CalciteSchema.TableEntry> starTables =
         Schemas.getStarTables(schema.root());
     if (starTables.isEmpty()) {
       // Don't waste effort converting to leaf-join form.
@@ -107,7 +123,7 @@ class OptiqMaterializer extends OptiqPrepareImpl.OptiqPreparingStmt {
     final List<Callback> list = Lists.newArrayList();
     final RelNode rel2 =
         RelOptMaterialization.toLeafJoinForm(queryRel);
-    for (OptiqSchema.TableEntry starTable : starTables) {
+    for (CalciteSchema.TableEntry starTable : starTables) {
       final Table table = starTable.getTable();
       assert table instanceof StarTable;
       RelOptTableImpl starRelOptTable =
@@ -125,40 +141,40 @@ class OptiqMaterializer extends OptiqPrepareImpl.OptiqPreparingStmt {
   /** Implementation of {@link RelShuttle} that returns each relational
    * expression unchanged. It does not visit children. */
   static class RelNullShuttle implements RelShuttle {
-    public RelNode visit(TableAccessRelBase scan) {
+    public RelNode visit(TableScan scan) {
       return scan;
     }
-    public RelNode visit(TableFunctionRelBase scan) {
+    public RelNode visit(TableFunctionScan scan) {
       return scan;
     }
-    public RelNode visit(ValuesRel values) {
+    public RelNode visit(LogicalValues values) {
       return values;
     }
-    public RelNode visit(FilterRel filter) {
+    public RelNode visit(LogicalFilter filter) {
       return filter;
     }
-    public RelNode visit(ProjectRel project) {
+    public RelNode visit(LogicalProject project) {
       return project;
     }
-    public RelNode visit(JoinRel join) {
+    public RelNode visit(LogicalJoin join) {
       return join;
     }
-    public RelNode visit(CorrelatorRel correlator) {
+    public RelNode visit(Correlator correlator) {
       return correlator;
     }
-    public RelNode visit(UnionRel union) {
+    public RelNode visit(LogicalUnion union) {
       return union;
     }
-    public RelNode visit(IntersectRel intersect) {
+    public RelNode visit(LogicalIntersect intersect) {
       return intersect;
     }
-    public RelNode visit(MinusRel minus) {
+    public RelNode visit(LogicalMinus minus) {
       return minus;
     }
-    public RelNode visit(AggregateRel aggregate) {
+    public RelNode visit(LogicalAggregate aggregate) {
       return aggregate;
     }
-    public RelNode visit(SortRel sort) {
+    public RelNode visit(Sort sort) {
       return sort;
     }
     public RelNode visit(RelNode other) {
@@ -169,11 +185,11 @@ class OptiqMaterializer extends OptiqPrepareImpl.OptiqPreparingStmt {
   /** Called when we discover a star table that matches. */
   static class Callback {
     public final RelNode rel;
-    public final OptiqSchema.TableEntry starTable;
+    public final CalciteSchema.TableEntry starTable;
     public final RelOptTableImpl starRelOptTable;
 
     Callback(RelNode rel,
-        OptiqSchema.TableEntry starTable,
+        CalciteSchema.TableEntry starTable,
         RelOptTableImpl starRelOptTable) {
       this.rel = rel;
       this.starTable = starTable;
@@ -182,4 +198,4 @@ class OptiqMaterializer extends OptiqPrepareImpl.OptiqPreparingStmt {
   }
 }
 
-// End OptiqMaterializer.java
+// End CalciteMaterializer.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java b/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
index 130638e..bdfba29 100644
--- a/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
+++ b/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
@@ -14,48 +14,106 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.prepare;
-
-import net.hydromatic.avatica.AvaticaParameter;
-import net.hydromatic.avatica.ColumnMetaData;
-import net.hydromatic.avatica.Helper;
-
-import net.hydromatic.linq4j.*;
-import net.hydromatic.linq4j.expressions.*;
-import net.hydromatic.linq4j.function.Function1;
-
-import net.hydromatic.optiq.*;
-import net.hydromatic.optiq.config.OptiqConnectionConfig;
-import net.hydromatic.optiq.impl.java.JavaTypeFactory;
-import net.hydromatic.optiq.jdbc.OptiqPrepare;
-import net.hydromatic.optiq.jdbc.OptiqSchema;
-import net.hydromatic.optiq.materialize.MaterializationService;
-import net.hydromatic.optiq.rules.java.*;
-import net.hydromatic.optiq.runtime.*;
-import net.hydromatic.optiq.server.OptiqServerStatement;
-import net.hydromatic.optiq.tools.Frameworks;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.rel.rules.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.relopt.hep.*;
-import org.eigenbase.relopt.volcano.VolcanoPlanner;
-import org.eigenbase.reltype.*;
-import org.eigenbase.rex.RexBuilder;
-import org.eigenbase.rex.RexNode;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.fun.SqlStdOperatorTable;
-import org.eigenbase.sql.parser.SqlParseException;
-import org.eigenbase.sql.parser.SqlParser;
-import org.eigenbase.sql.parser.impl.SqlParserImpl;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.sql.util.ChainedSqlOperatorTable;
-import org.eigenbase.sql.validate.*;
-import org.eigenbase.sql2rel.SqlToRelConverter;
-import org.eigenbase.sql2rel.StandardConvertletTable;
-import org.eigenbase.util.Util;
-
-import com.google.common.collect.*;
+package org.apache.calcite.prepare;
+
+import org.apache.calcite.DataContext;
+import org.apache.calcite.adapter.enumerable.EnumerableConvention;
+import org.apache.calcite.adapter.enumerable.EnumerableRel;
+import org.apache.calcite.adapter.enumerable.EnumerableRelImplementor;
+import org.apache.calcite.adapter.enumerable.EnumerableRules;
+import org.apache.calcite.adapter.enumerable.RexToLixTranslator;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.avatica.AvaticaParameter;
+import org.apache.calcite.avatica.ColumnMetaData;
+import org.apache.calcite.avatica.Helper;
+import org.apache.calcite.config.CalciteConnectionConfig;
+import org.apache.calcite.jdbc.CalcitePrepare;
+import org.apache.calcite.jdbc.CalciteSchema;
+import org.apache.calcite.linq4j.Enumerable;
+import org.apache.calcite.linq4j.Linq4j;
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.linq4j.Queryable;
+import org.apache.calcite.linq4j.function.Function1;
+import org.apache.calcite.linq4j.tree.BinaryExpression;
+import org.apache.calcite.linq4j.tree.BlockStatement;
+import org.apache.calcite.linq4j.tree.Blocks;
+import org.apache.calcite.linq4j.tree.ClassDeclaration;
+import org.apache.calcite.linq4j.tree.ConstantExpression;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.linq4j.tree.MemberExpression;
+import org.apache.calcite.linq4j.tree.MethodCallExpression;
+import org.apache.calcite.linq4j.tree.NewExpression;
+import org.apache.calcite.linq4j.tree.ParameterExpression;
+import org.apache.calcite.materialize.MaterializationService;
+import org.apache.calcite.plan.Contexts;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.ConventionTraitDef;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCostFactory;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelOptQuery;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.hep.HepPlanner;
+import org.apache.calcite.plan.hep.HepProgramBuilder;
+import org.apache.calcite.plan.volcano.VolcanoPlanner;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.rules.AggregateExpandDistinctAggregatesRule;
+import org.apache.calcite.rel.rules.AggregateReduceFunctionsRule;
+import org.apache.calcite.rel.rules.AggregateStarTableRule;
+import org.apache.calcite.rel.rules.FilterAggregateTransposeRule;
+import org.apache.calcite.rel.rules.FilterJoinRule;
+import org.apache.calcite.rel.rules.FilterProjectTransposeRule;
+import org.apache.calcite.rel.rules.FilterTableRule;
+import org.apache.calcite.rel.rules.JoinAssociateRule;
+import org.apache.calcite.rel.rules.JoinCommuteRule;
+import org.apache.calcite.rel.rules.JoinPushThroughJoinRule;
+import org.apache.calcite.rel.rules.ProjectFilterTransposeRule;
+import org.apache.calcite.rel.rules.ProjectMergeRule;
+import org.apache.calcite.rel.rules.ProjectTableRule;
+import org.apache.calcite.rel.rules.ReduceExpressionsRule;
+import org.apache.calcite.rel.rules.SortProjectTransposeRule;
+import org.apache.calcite.rel.rules.TableScanRule;
+import org.apache.calcite.rel.rules.ValuesReduceRule;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeFactoryImpl;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.runtime.Bindable;
+import org.apache.calcite.runtime.Hook;
+import org.apache.calcite.runtime.Spaces;
+import org.apache.calcite.runtime.Typed;
+import org.apache.calcite.runtime.Utilities;
+import org.apache.calcite.schema.Schemas;
+import org.apache.calcite.server.CalciteServerStatement;
+import org.apache.calcite.sql.SqlBinaryOperator;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.parser.SqlParseException;
+import org.apache.calcite.sql.parser.SqlParser;
+import org.apache.calcite.sql.parser.impl.SqlParserImpl;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.util.ChainedSqlOperatorTable;
+import org.apache.calcite.sql.validate.SqlConformance;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql.validate.SqlValidatorImpl;
+import org.apache.calcite.sql2rel.SqlToRelConverter;
+import org.apache.calcite.sql2rel.StandardConvertletTable;
+import org.apache.calcite.tools.Frameworks;
+import org.apache.calcite.util.Util;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Maps;
 
 import org.codehaus.commons.compiler.CompileException;
 import org.codehaus.commons.compiler.CompilerFactoryFactory;
@@ -69,7 +127,11 @@ import java.io.StringReader;
 import java.lang.reflect.Type;
 import java.math.BigDecimal;
 import java.sql.DatabaseMetaData;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
 
 /**
  * Shit just got real.
@@ -78,7 +140,7 @@ import java.util.*;
  * and server can fine-tune preferences. However, this class and its methods are
  * subject to change without notice.</p>
  */
-public class OptiqPrepareImpl implements OptiqPrepare {
+public class CalcitePrepareImpl implements CalcitePrepare {
 
   public static final boolean DEBUG =
       "true".equals(System.getProperties().getProperty("calcite.debug"));
@@ -104,43 +166,43 @@ public class OptiqPrepareImpl implements OptiqPrepare {
 
   private static final List<RelOptRule> DEFAULT_RULES =
       ImmutableList.of(
-          JavaRules.ENUMERABLE_JOIN_RULE,
-          JavaRules.ENUMERABLE_SEMI_JOIN_RULE,
-          JavaRules.ENUMERABLE_PROJECT_RULE,
-          JavaRules.ENUMERABLE_FILTER_RULE,
-          JavaRules.ENUMERABLE_AGGREGATE_RULE,
-          JavaRules.ENUMERABLE_SORT_RULE,
-          JavaRules.ENUMERABLE_LIMIT_RULE,
-          JavaRules.ENUMERABLE_COLLECT_RULE,
-          JavaRules.ENUMERABLE_UNCOLLECT_RULE,
-          JavaRules.ENUMERABLE_UNION_RULE,
-          JavaRules.ENUMERABLE_INTERSECT_RULE,
-          JavaRules.ENUMERABLE_MINUS_RULE,
-          JavaRules.ENUMERABLE_TABLE_MODIFICATION_RULE,
-          JavaRules.ENUMERABLE_VALUES_RULE,
-          JavaRules.ENUMERABLE_WINDOW_RULE,
-          JavaRules.ENUMERABLE_ONE_ROW_RULE,
-          JavaRules.ENUMERABLE_EMPTY_RULE,
-          JavaRules.ENUMERABLE_TABLE_FUNCTION_RULE,
+          EnumerableRules.ENUMERABLE_JOIN_RULE,
+          EnumerableRules.ENUMERABLE_SEMI_JOIN_RULE,
+          EnumerableRules.ENUMERABLE_PROJECT_RULE,
+          EnumerableRules.ENUMERABLE_FILTER_RULE,
+          EnumerableRules.ENUMERABLE_AGGREGATE_RULE,
+          EnumerableRules.ENUMERABLE_SORT_RULE,
+          EnumerableRules.ENUMERABLE_LIMIT_RULE,
+          EnumerableRules.ENUMERABLE_COLLECT_RULE,
+          EnumerableRules.ENUMERABLE_UNCOLLECT_RULE,
+          EnumerableRules.ENUMERABLE_UNION_RULE,
+          EnumerableRules.ENUMERABLE_INTERSECT_RULE,
+          EnumerableRules.ENUMERABLE_MINUS_RULE,
+          EnumerableRules.ENUMERABLE_TABLE_MODIFICATION_RULE,
+          EnumerableRules.ENUMERABLE_VALUES_RULE,
+          EnumerableRules.ENUMERABLE_WINDOW_RULE,
+          EnumerableRules.ENUMERABLE_ONE_ROW_RULE,
+          EnumerableRules.ENUMERABLE_EMPTY_RULE,
+          EnumerableRules.ENUMERABLE_TABLE_FUNCTION_SCAN_RULE,
           AggregateStarTableRule.INSTANCE,
           AggregateStarTableRule.INSTANCE2,
-          TableAccessRule.INSTANCE,
+          TableScanRule.INSTANCE,
           COMMUTE
-              ? CommutativeJoinRule.INSTANCE
-              : MergeProjectRule.INSTANCE,
+              ? JoinAssociateRule.INSTANCE
+              : ProjectMergeRule.INSTANCE,
           FilterTableRule.INSTANCE,
           ProjectTableRule.INSTANCE,
           ProjectTableRule.INSTANCE2,
-          PushProjectPastFilterRule.INSTANCE,
-          PushFilterPastProjectRule.INSTANCE,
-          PushFilterPastJoinRule.FILTER_ON_JOIN,
-          RemoveDistinctAggregateRule.INSTANCE,
-          ReduceAggregatesRule.INSTANCE,
+          ProjectFilterTransposeRule.INSTANCE,
+          FilterProjectTransposeRule.INSTANCE,
+          FilterJoinRule.FILTER_ON_JOIN,
+          AggregateExpandDistinctAggregatesRule.INSTANCE,
+          AggregateReduceFunctionsRule.INSTANCE,
           FilterAggregateTransposeRule.INSTANCE,
-          SwapJoinRule.INSTANCE,
-          PushJoinThroughJoinRule.RIGHT,
-          PushJoinThroughJoinRule.LEFT,
-          PushSortPastProjectRule.INSTANCE);
+          JoinCommuteRule.INSTANCE,
+          JoinPushThroughJoinRule.RIGHT,
+          JoinPushThroughJoinRule.LEFT,
+          SortProjectTransposeRule.INSTANCE);
 
   private static final List<RelOptRule> CONSTANT_REDUCTION_RULES =
       ImmutableList.of(
@@ -148,11 +210,11 @@ public class OptiqPrepareImpl implements OptiqPrepare {
           ReduceExpressionsRule.FILTER_INSTANCE,
           ReduceExpressionsRule.CALC_INSTANCE,
           ReduceExpressionsRule.JOIN_INSTANCE,
-          ReduceValuesRule.FILTER_INSTANCE,
-          ReduceValuesRule.PROJECT_FILTER_INSTANCE,
-          ReduceValuesRule.PROJECT_INSTANCE);
+          ValuesReduceRule.FILTER_INSTANCE,
+          ValuesReduceRule.PROJECT_FILTER_INSTANCE,
+          ValuesReduceRule.PROJECT_INSTANCE);
 
-  public OptiqPrepareImpl() {
+  public CalcitePrepareImpl() {
   }
 
   public ParseResult parse(
@@ -167,8 +229,8 @@ public class OptiqPrepareImpl implements OptiqPrepare {
   /** Shared implementation for {@link #parse} and {@link #convert}. */
   private ParseResult parse_(Context context, String sql, boolean convert) {
     final JavaTypeFactory typeFactory = context.getTypeFactory();
-    OptiqCatalogReader catalogReader =
-        new OptiqCatalogReader(
+    CalciteCatalogReader catalogReader =
+        new CalciteCatalogReader(
             context.getRootSchema(),
             context.config().caseSensitive(),
             context.getDefaultSchemaPath(),
@@ -181,15 +243,15 @@ public class OptiqPrepareImpl implements OptiqPrepare {
       throw new RuntimeException("parse failed", e);
     }
     final SqlValidator validator =
-        new OptiqSqlValidator(
+        new CalciteSqlValidator(
             SqlStdOperatorTable.instance(), catalogReader, typeFactory);
     SqlNode sqlNode1 = validator.validate(sqlNode);
     if (!convert) {
       return new ParseResult(this, validator, sql, sqlNode1,
           validator.getValidatedNodeType(sqlNode1));
     }
-    final OptiqPreparingStmt preparingStmt =
-        new OptiqPreparingStmt(
+    final CalcitePreparingStmt preparingStmt =
+        new CalcitePreparingStmt(
             context,
             catalogReader,
             typeFactory,
@@ -215,7 +277,7 @@ public class OptiqPrepareImpl implements OptiqPrepare {
    * complex planner for complex and costly queries.</p>
    *
    * <p>The default implementation returns a factory that calls
-   * {@link #createPlanner(net.hydromatic.optiq.jdbc.OptiqPrepare.Context)}.</p>
+   * {@link #createPlanner(org.apache.calcite.jdbc.CalcitePrepare.Context)}.</p>
    */
   protected List<Function1<Context, RelOptPlanner>> createPlannerFactories() {
     return Collections.<Function1<Context, RelOptPlanner>>singletonList(
@@ -228,15 +290,15 @@ public class OptiqPrepareImpl implements OptiqPrepare {
 
   /** Creates a query planner and initializes it with a default set of
    * rules. */
-  protected RelOptPlanner createPlanner(OptiqPrepare.Context prepareContext) {
+  protected RelOptPlanner createPlanner(CalcitePrepare.Context prepareContext) {
     return createPlanner(prepareContext, null, null);
   }
 
   /** Creates a query planner and initializes it with a default set of
    * rules. */
   protected RelOptPlanner createPlanner(
-      final OptiqPrepare.Context prepareContext,
-      org.eigenbase.relopt.Context externalContext,
+      final CalcitePrepare.Context prepareContext,
+      org.apache.calcite.plan.Context externalContext,
       RelOptCostFactory costFactory) {
     if (externalContext == null) {
       externalContext = Contexts.withConfig(prepareContext.config());
@@ -301,8 +363,8 @@ public class OptiqPrepareImpl implements OptiqPrepare {
       return simplePrepare(context, sql);
     }
     final JavaTypeFactory typeFactory = context.getTypeFactory();
-    OptiqCatalogReader catalogReader =
-        new OptiqCatalogReader(
+    CalciteCatalogReader catalogReader =
+        new CalciteCatalogReader(
             context.getRootSchema(),
             context.config().caseSensitive(),
             context.getDefaultSchemaPath(),
@@ -319,8 +381,7 @@ public class OptiqPrepareImpl implements OptiqPrepare {
         throw new AssertionError("factory returned null planner");
       }
       try {
-        return prepare2_(
-            context, sql, queryable, elementType, maxRowCount,
+        return prepare2_(context, sql, queryable, elementType, maxRowCount,
             catalogReader, planner);
       } catch (RelOptPlanner.CannotPlanException e) {
         exception = e;
@@ -361,7 +422,7 @@ public class OptiqPrepareImpl implements OptiqPrepare {
       Queryable<T> queryable,
       Type elementType,
       int maxRowCount,
-      OptiqCatalogReader catalogReader,
+      CalciteCatalogReader catalogReader,
       RelOptPlanner planner) {
     final JavaTypeFactory typeFactory = context.getTypeFactory();
     final EnumerableRel.Prefer prefer;
@@ -370,8 +431,8 @@ public class OptiqPrepareImpl implements OptiqPrepare {
     } else {
       prefer = EnumerableRel.Prefer.CUSTOM;
     }
-    final OptiqPreparingStmt preparingStmt =
-        new OptiqPreparingStmt(
+    final CalcitePreparingStmt preparingStmt =
+        new CalcitePreparingStmt(
             context,
             catalogReader,
             typeFactory,
@@ -384,7 +445,7 @@ public class OptiqPrepareImpl implements OptiqPrepare {
     final Prepare.PreparedResult preparedResult;
     if (sql != null) {
       assert queryable == null;
-      final OptiqConnectionConfig config = context.config();
+      final CalciteConnectionConfig config = context.config();
       SqlParser parser = SqlParser.create(SqlParserImpl.FACTORY, sql,
           config.quoting(), config.unquotedCasing(), config.quotedCasing());
       SqlNode sqlNode;
@@ -397,12 +458,12 @@ public class OptiqPrepareImpl implements OptiqPrepare {
 
       Hook.PARSE_TREE.run(new Object[] {sql, sqlNode});
 
-      final OptiqSchema rootSchema = context.getRootSchema();
+      final CalciteSchema rootSchema = context.getRootSchema();
       final ChainedSqlOperatorTable opTab =
           new ChainedSqlOperatorTable(
               ImmutableList.of(SqlStdOperatorTable.instance(), catalogReader));
       final SqlValidator validator =
-          new OptiqSqlValidator(opTab, catalogReader, typeFactory);
+          new CalciteSqlValidator(opTab, catalogReader, typeFactory);
       validator.setIdentifierExpansion(true);
 
       final List<Prepare.Materialization> materializations =
@@ -412,7 +473,7 @@ public class OptiqPrepareImpl implements OptiqPrepare {
       for (Prepare.Materialization materialization : materializations) {
         populateMaterializations(context, planner, materialization);
       }
-      final List<OptiqSchema.LatticeEntry> lattices =
+      final List<CalciteSchema.LatticeEntry> lattices =
           Schemas.getLatticeEntries(rootSchema);
       preparedResult = preparingStmt.prepareSql(
           sqlNode, Object.class, validator, true, materializations, lattices);
@@ -573,15 +634,15 @@ public class OptiqPrepareImpl implements OptiqPrepare {
     // REVIEW: initialize queryRel and tableRel inside MaterializationService,
     // not here?
     try {
-      final OptiqSchema schema = materialization.materializedTable.schema;
-      OptiqCatalogReader catalogReader =
-          new OptiqCatalogReader(
+      final CalciteSchema schema = materialization.materializedTable.schema;
+      CalciteCatalogReader catalogReader =
+          new CalciteCatalogReader(
               schema.root(),
               context.config().caseSensitive(),
               Util.skipLast(materialization.materializedTable.path()),
               context.getTypeFactory());
-      final OptiqMaterializer materializer =
-          new OptiqMaterializer(context, catalogReader, schema, planner);
+      final CalciteMaterializer materializer =
+          new CalciteMaterializer(context, catalogReader, schema, planner);
       materializer.populate(materialization);
     } catch (Exception e) {
       throw new RuntimeException("While populating materialization "
@@ -599,13 +660,13 @@ public class OptiqPrepareImpl implements OptiqPrepare {
   }
 
   /** Executes a prepare action. */
-  public <R> R perform(OptiqServerStatement statement,
+  public <R> R perform(CalciteServerStatement statement,
       Frameworks.PrepareAction<R> action) {
-    final OptiqPrepare.Context prepareContext =
+    final CalcitePrepare.Context prepareContext =
         statement.createPrepareContext();
     final JavaTypeFactory typeFactory = prepareContext.getTypeFactory();
-    OptiqCatalogReader catalogReader =
-        new OptiqCatalogReader(prepareContext.getRootSchema(),
+    CalciteCatalogReader catalogReader =
+        new CalciteCatalogReader(prepareContext.getRootSchema(),
             prepareContext.config().caseSensitive(),
             prepareContext.getDefaultSchemaPath(),
             typeFactory);
@@ -621,11 +682,12 @@ public class OptiqPrepareImpl implements OptiqPrepare {
         prepareContext.getRootSchema().plus(), statement);
   }
 
-  static class OptiqPreparingStmt extends Prepare
+  /** Holds state for the process of preparing a SQL statement. */
+  static class CalcitePreparingStmt extends Prepare
       implements RelOptTable.ViewExpander {
     private final RelOptPlanner planner;
     private final RexBuilder rexBuilder;
-    protected final OptiqSchema schema;
+    protected final CalciteSchema schema;
     protected final RelDataTypeFactory typeFactory;
     private final EnumerableRel.Prefer prefer;
     private final Map<String, Object> internalParameters =
@@ -633,10 +695,10 @@ public class OptiqPrepareImpl implements OptiqPrepare {
     private int expansionDepth;
     private SqlValidator sqlValidator;
 
-    public OptiqPreparingStmt(Context context,
+    public CalcitePreparingStmt(Context context,
         CatalogReader catalogReader,
         RelDataTypeFactory typeFactory,
-        OptiqSchema schema,
+        CalciteSchema schema,
         EnumerableRel.Prefer prefer,
         RelOptPlanner planner,
         Convention resultConvention) {
@@ -648,8 +710,7 @@ public class OptiqPrepareImpl implements OptiqPrepare {
       this.rexBuilder = new RexBuilder(typeFactory);
     }
 
-    @Override
-    protected void init(Class runtimeContextClass) {
+    @Override protected void init(Class runtimeContextClass) {
     }
 
     public PreparedResult prepareQueryable(
@@ -665,7 +726,7 @@ public class OptiqPrepareImpl implements OptiqPrepare {
               rexBuilder.getTypeFactory(), rexBuilder);
 
       RelNode rootRel =
-          new LixToRelTranslator(cluster, OptiqPreparingStmt.this)
+          new LixToRelTranslator(cluster, CalcitePreparingStmt.this)
               .translate(queryable);
 
       if (timingTracer != null) {
@@ -685,7 +746,7 @@ public class OptiqPrepareImpl implements OptiqPrepare {
       rootRel = trimUnusedFields(rootRel);
 
       final List<Materialization> materializations = ImmutableList.of();
-      final List<OptiqSchema.LatticeEntry> lattices = ImmutableList.of();
+      final List<CalciteSchema.LatticeEntry> lattices = ImmutableList.of();
       rootRel = optimize(resultType, rootRel, materializations, lattices);
 
       if (timingTracer != null) {
@@ -698,8 +759,7 @@ public class OptiqPrepareImpl implements OptiqPrepare {
           SqlKind.SELECT);
     }
 
-    @Override
-    protected SqlToRelConverter getSqlToRelConverter(
+    @Override protected SqlToRelConverter getSqlToRelConverter(
         SqlValidator validator,
         CatalogReader catalogReader) {
       SqlToRelConverter sqlToRelConverter =
@@ -710,19 +770,16 @@ public class OptiqPrepareImpl implements OptiqPrepare {
       return sqlToRelConverter;
     }
 
-    @Override
-    protected EnumerableRelImplementor getRelImplementor(
+    @Override protected EnumerableRelImplementor getRelImplementor(
         RexBuilder rexBuilder) {
       return new EnumerableRelImplementor(rexBuilder, internalParameters);
     }
 
-    @Override
-    protected boolean shouldAlwaysWriteJavaFile() {
+    @Override protected boolean shouldAlwaysWriteJavaFile() {
       return false;
     }
 
-    @Override
-    public RelNode flattenTypes(
+    @Override public RelNode flattenTypes(
         RelNode rootRel,
         boolean restructure) {
       final SparkHandler spark = context.spark();
@@ -771,27 +828,24 @@ public class OptiqPrepareImpl implements OptiqPrepare {
           rexBuilder.getTypeFactory(), SqlConformance.DEFAULT) { };
     }
 
-    @Override
-    protected SqlValidator getSqlValidator() {
+    @Override protected SqlValidator getSqlValidator() {
       if (sqlValidator == null) {
         sqlValidator = createSqlValidator(catalogReader);
       }
       return sqlValidator;
     }
 
-    @Override
-    protected PreparedResult createPreparedExplanation(
+    @Override protected PreparedResult createPreparedExplanation(
         RelDataType resultType,
         RelDataType parameterRowType,
         RelNode rootRel,
         boolean explainAsXml,
         SqlExplainLevel detailLevel) {
-      return new OptiqPreparedExplain(
+      return new CalcitePreparedExplain(
           resultType, parameterRowType, rootRel, explainAsXml, detailLevel);
     }
 
-    @Override
-    protected PreparedResult implement(
+    @Override protected PreparedResult implement(
         RelDataType rowType,
         RelNode rootRel,
         SqlKind sqlKind) {
@@ -897,8 +951,9 @@ public class OptiqPrepareImpl implements OptiqPrepare {
     }
   }
 
-  private static class OptiqPreparedExplain extends Prepare.PreparedExplain {
-    public OptiqPreparedExplain(
+  /** An {@code EXPLAIN} statement, prepared and ready to execute. */
+  private static class CalcitePreparedExplain extends Prepare.PreparedExplain {
+    public CalcitePreparedExplain(
         RelDataType resultType,
         RelDataType parameterRowType,
         RelNode rootRel,
@@ -907,7 +962,6 @@ public class OptiqPrepareImpl implements OptiqPrepare {
       super(resultType, parameterRowType, rootRel, explainAsXml, detailLevel);
     }
 
-    @Override
     public Bindable getBindable() {
       final String explanation = getCode();
       return new Bindable() {
@@ -918,14 +972,16 @@ public class OptiqPrepareImpl implements OptiqPrepare {
     }
   }
 
+  /** Translator from Java AST to {@link RexNode}. */
   interface ScalarTranslator {
     RexNode toRex(BlockStatement statement);
     List<RexNode> toRexList(BlockStatement statement);
     RexNode toRex(Expression expression);
-    ScalarTranslator bind(
-        List<ParameterExpression> parameterList, List<RexNode> values);
+    ScalarTranslator bind(List<ParameterExpression> parameterList,
+        List<RexNode> values);
   }
 
+  /** Basic translator. */
   static class EmptyScalarTranslator implements ScalarTranslator {
     private final RexBuilder rexBuilder;
 
@@ -1047,6 +1103,7 @@ public class OptiqPrepareImpl implements OptiqPrepare {
     }
   }
 
+  /** Translator that looks for parameters. */
   private static class LambdaScalarTranslator extends EmptyScalarTranslator {
     private final List<ParameterExpression> parameterList;
     private final List<RexNode> values;
@@ -1070,4 +1127,4 @@ public class OptiqPrepareImpl implements OptiqPrepare {
   }
 }
 
-// End OptiqPrepareImpl.java
+// End CalcitePrepareImpl.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/prepare/CalciteSqlValidator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/prepare/CalciteSqlValidator.java b/core/src/main/java/org/apache/calcite/prepare/CalciteSqlValidator.java
index 6236a29..63670ad 100644
--- a/core/src/main/java/org/apache/calcite/prepare/CalciteSqlValidator.java
+++ b/core/src/main/java/org/apache/calcite/prepare/CalciteSqlValidator.java
@@ -14,36 +14,33 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.prepare;
+package org.apache.calcite.prepare;
 
-import net.hydromatic.optiq.impl.java.JavaTypeFactory;
-
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.sql.SqlInsert;
-import org.eigenbase.sql.SqlOperatorTable;
-import org.eigenbase.sql.validate.SqlConformance;
-import org.eigenbase.sql.validate.SqlValidatorImpl;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlInsert;
+import org.apache.calcite.sql.SqlOperatorTable;
+import org.apache.calcite.sql.validate.SqlConformance;
+import org.apache.calcite.sql.validate.SqlValidatorImpl;
 
 /** Validator. */
-class OptiqSqlValidator extends SqlValidatorImpl {
-  public OptiqSqlValidator(
+class CalciteSqlValidator extends SqlValidatorImpl {
+  public CalciteSqlValidator(
       SqlOperatorTable opTab,
-      OptiqCatalogReader catalogReader,
+      CalciteCatalogReader catalogReader,
       JavaTypeFactory typeFactory) {
     super(opTab, catalogReader, typeFactory, SqlConformance.DEFAULT);
   }
 
-  @Override
-  protected RelDataType getLogicalSourceRowType(
+  @Override protected RelDataType getLogicalSourceRowType(
       RelDataType sourceRowType, SqlInsert insert) {
     return ((JavaTypeFactory) typeFactory).toSql(sourceRowType);
   }
 
-  @Override
-  protected RelDataType getLogicalTargetRowType(
+  @Override protected RelDataType getLogicalTargetRowType(
       RelDataType targetRowType, SqlInsert insert) {
     return ((JavaTypeFactory) typeFactory).toSql(targetRowType);
   }
 }
 
-// End OptiqSqlValidator.java
+// End CalciteSqlValidator.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/prepare/LixToRelTranslator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/prepare/LixToRelTranslator.java b/core/src/main/java/org/apache/calcite/prepare/LixToRelTranslator.java
index dde0295..35b43cb 100644
--- a/core/src/main/java/org/apache/calcite/prepare/LixToRelTranslator.java
+++ b/core/src/main/java/org/apache/calcite/prepare/LixToRelTranslator.java
@@ -14,20 +14,27 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.prepare;
-
-import net.hydromatic.linq4j.Queryable;
-import net.hydromatic.linq4j.expressions.*;
-
-import net.hydromatic.optiq.*;
-import net.hydromatic.optiq.impl.java.JavaTypeFactory;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.RelOptCluster;
-import org.eigenbase.relopt.RelOptTable;
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.rex.RexBuilder;
-import org.eigenbase.rex.RexNode;
+package org.apache.calcite.prepare;
+
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.linq4j.Queryable;
+import org.apache.calcite.linq4j.tree.Blocks;
+import org.apache.calcite.linq4j.tree.ConstantExpression;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.FunctionExpression;
+import org.apache.calcite.linq4j.tree.MethodCallExpression;
+import org.apache.calcite.linq4j.tree.NewExpression;
+import org.apache.calcite.linq4j.tree.Types;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.logical.LogicalFilter;
+import org.apache.calcite.rel.logical.LogicalProject;
+import org.apache.calcite.rel.logical.LogicalTableScan;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.util.BuiltInMethod;
 
 import com.google.common.collect.ImmutableList;
 
@@ -70,7 +77,7 @@ class LixToRelTranslator implements RelOptTable.ToRelContext {
   public RelNode translate(Expression expression) {
     if (expression instanceof MethodCallExpression) {
       final MethodCallExpression call = (MethodCallExpression) expression;
-      BuiltinMethod method = BuiltinMethod.MAP.get(call.method);
+      BuiltInMethod method = BuiltInMethod.MAP.get(call.method);
       if (method == null) {
         throw new UnsupportedOperationException(
             "unknown method " + call.method);
@@ -79,18 +86,18 @@ class LixToRelTranslator implements RelOptTable.ToRelContext {
       switch (method) {
       case SELECT:
         child = translate(call.targetExpression);
-        return new ProjectRel(
+        return new LogicalProject(
             cluster,
             child,
             toRex(
                 child,
                 (FunctionExpression) call.expressions.get(0)),
             null,
-            ProjectRel.Flags.BOXED);
+            LogicalProject.Flags.BOXED);
 
       case WHERE:
         child = translate(call.targetExpression);
-        return new FilterRel(
+        return new LogicalFilter(
             cluster,
             child,
             toRex(
@@ -98,7 +105,7 @@ class LixToRelTranslator implements RelOptTable.ToRelContext {
                 child));
 
       case AS_QUERYABLE:
-        return new TableAccessRel(
+        return new LogicalTableScan(
             cluster,
             RelOptTableImpl.create(
                 null,
@@ -109,7 +116,7 @@ class LixToRelTranslator implements RelOptTable.ToRelContext {
                 call.targetExpression));
 
       case SCHEMA_GET_TABLE:
-        return new TableAccessRel(
+        return new LogicalTableScan(
             cluster,
             RelOptTableImpl.create(
                 null,
@@ -133,8 +140,8 @@ class LixToRelTranslator implements RelOptTable.ToRelContext {
     List<RexNode> list =
         Collections.singletonList(
             rexBuilder.makeRangeReference(child));
-    OptiqPrepareImpl.ScalarTranslator translator =
-        OptiqPrepareImpl.EmptyScalarTranslator
+    CalcitePrepareImpl.ScalarTranslator translator =
+        CalcitePrepareImpl.EmptyScalarTranslator
             .empty(rexBuilder)
             .bind(expression.parameterList, list);
     final List<RexNode> rexList = new ArrayList<RexNode>();
@@ -163,7 +170,7 @@ class LixToRelTranslator implements RelOptTable.ToRelContext {
     for (RelNode input : inputs) {
       list.add(rexBuilder.makeRangeReference(input));
     }
-    return OptiqPrepareImpl.EmptyScalarTranslator.empty(rexBuilder)
+    return CalcitePrepareImpl.EmptyScalarTranslator.empty(rexBuilder)
         .bind(expression.parameterList, list)
         .toRexList(expression.body);
   }
@@ -176,7 +183,7 @@ class LixToRelTranslator implements RelOptTable.ToRelContext {
     for (RelNode input : inputs) {
       list.add(rexBuilder.makeRangeReference(input));
     }
-    return OptiqPrepareImpl.EmptyScalarTranslator.empty(rexBuilder)
+    return CalcitePrepareImpl.EmptyScalarTranslator.empty(rexBuilder)
         .bind(expression.parameterList, list)
         .toRex(expression.body);
   }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/prepare/PlannerImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/prepare/PlannerImpl.java b/core/src/main/java/org/apache/calcite/prepare/PlannerImpl.java
index 4dc5124..278ba11 100644
--- a/core/src/main/java/org/apache/calcite/prepare/PlannerImpl.java
+++ b/core/src/main/java/org/apache/calcite/prepare/PlannerImpl.java
@@ -14,35 +14,43 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.prepare;
-
-import net.hydromatic.optiq.SchemaPlus;
-import net.hydromatic.optiq.config.Lex;
-import net.hydromatic.optiq.impl.java.JavaTypeFactory;
-import net.hydromatic.optiq.jdbc.OptiqSchema;
-import net.hydromatic.optiq.tools.*;
-
-import org.eigenbase.rel.RelNode;
-import org.eigenbase.relopt.*;
-import org.eigenbase.relopt.RelOptTable.ViewExpander;
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.rex.RexBuilder;
-import org.eigenbase.sql.SqlNode;
-import org.eigenbase.sql.SqlOperatorTable;
-import org.eigenbase.sql.parser.SqlParseException;
-import org.eigenbase.sql.parser.SqlParser;
-import org.eigenbase.sql.parser.SqlParserImplFactory;
-import org.eigenbase.sql.validate.SqlValidator;
-import org.eigenbase.sql2rel.RelDecorrelator;
-import org.eigenbase.sql2rel.SqlRexConvertletTable;
-import org.eigenbase.sql2rel.SqlToRelConverter;
-import org.eigenbase.util.Util;
+package org.apache.calcite.prepare;
+
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.config.Lex;
+import org.apache.calcite.jdbc.CalciteSchema;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelOptSchema;
+import org.apache.calcite.plan.RelOptTable.ViewExpander;
+import org.apache.calcite.plan.RelTraitDef;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperatorTable;
+import org.apache.calcite.sql.parser.SqlParseException;
+import org.apache.calcite.sql.parser.SqlParser;
+import org.apache.calcite.sql.parser.SqlParserImplFactory;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql2rel.RelDecorrelator;
+import org.apache.calcite.sql2rel.SqlRexConvertletTable;
+import org.apache.calcite.sql2rel.SqlToRelConverter;
+import org.apache.calcite.tools.FrameworkConfig;
+import org.apache.calcite.tools.Frameworks;
+import org.apache.calcite.tools.Planner;
+import org.apache.calcite.tools.Program;
+import org.apache.calcite.tools.RelConversionException;
+import org.apache.calcite.tools.ValidationException;
+import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
 
 import java.util.List;
 
-/** Implementation of {@link net.hydromatic.optiq.tools.Planner}. */
+/** Implementation of {@link org.apache.calcite.tools.Planner}. */
 public class PlannerImpl implements Planner {
   private final SqlOperatorTable operatorTable;
   private final ImmutableList<Program> programs;
@@ -69,14 +77,14 @@ public class PlannerImpl implements Planner {
   private RelOptPlanner planner;
 
   // set in STATE_4_VALIDATE
-  private OptiqSqlValidator validator;
+  private CalciteSqlValidator validator;
   private SqlNode validatedSqlNode;
 
   // set in STATE_5_CONVERT
   private RelNode rel;
 
   /** Creates a planner. Not a public API; call
-   * {@link net.hydromatic.optiq.tools.Frameworks#getPlanner} instead. */
+   * {@link org.apache.calcite.tools.Frameworks#getPlanner} instead. */
   public PlannerImpl(FrameworkConfig config) {
     this.config = config;
     this.defaultSchema = config.getDefaultSchema();
@@ -166,7 +174,7 @@ public class PlannerImpl implements Planner {
   public SqlNode validate(SqlNode sqlNode) throws ValidationException {
     ensure(State.STATE_3_PARSED);
     this.validator =
-        new OptiqSqlValidator(
+        new CalciteSqlValidator(
             operatorTable, createCatalogReader(), typeFactory);
     try {
       validatedSqlNode = validator.validate(sqlNode);
@@ -193,8 +201,8 @@ public class PlannerImpl implements Planner {
     return rel;
   }
 
-  /** Implements {@link org.eigenbase.relopt.RelOptTable.ViewExpander}
-   * interface for {@link net.hydromatic.optiq.tools.Planner}. */
+  /** Implements {@link org.apache.calcite.plan.RelOptTable.ViewExpander}
+   * interface for {@link org.apache.calcite.tools.Planner}. */
   public class ViewExpanderImpl implements ViewExpander {
     public RelNode expandView(RelDataType rowType, String queryString,
         List<String> schemaPath) {
@@ -207,9 +215,9 @@ public class PlannerImpl implements Planner {
         throw new RuntimeException("parse failed", e);
       }
 
-      final OptiqCatalogReader catalogReader =
+      final CalciteCatalogReader catalogReader =
           createCatalogReader().withSchemaPath(schemaPath);
-      final SqlValidator validator = new OptiqSqlValidator(operatorTable,
+      final SqlValidator validator = new CalciteSqlValidator(operatorTable,
           catalogReader, typeFactory);
       final SqlNode validatedSqlNode = validator.validate(sqlNode);
 
@@ -222,13 +230,13 @@ public class PlannerImpl implements Planner {
     }
   }
 
-  // OptiqCatalogReader is stateless; no need to store one
-  private OptiqCatalogReader createCatalogReader() {
+  // CalciteCatalogReader is stateless; no need to store one
+  private CalciteCatalogReader createCatalogReader() {
     SchemaPlus rootSchema = rootSchema(defaultSchema);
-    return new OptiqCatalogReader(
-        OptiqSchema.from(rootSchema),
+    return new CalciteCatalogReader(
+        CalciteSchema.from(rootSchema),
         caseSensitive,
-        OptiqSchema.from(defaultSchema).path(null),
+        CalciteSchema.from(defaultSchema).path(null),
         typeFactory);
   }
 
@@ -260,21 +268,18 @@ public class PlannerImpl implements Planner {
   /** Stage of a statement in the query-preparation lifecycle. */
   private enum State {
     STATE_0_CLOSED {
-      @Override
-      void from(PlannerImpl planner) {
+      @Override void from(PlannerImpl planner) {
         planner.close();
       }
     },
     STATE_1_RESET {
-      @Override
-      void from(PlannerImpl planner) {
+      @Override void from(PlannerImpl planner) {
         planner.ensure(STATE_0_CLOSED);
         planner.reset();
       }
     },
     STATE_2_READY {
-      @Override
-      void from(PlannerImpl planner) {
+      @Override void from(PlannerImpl planner) {
         STATE_1_RESET.from(planner);
         planner.ready();
       }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/prepare/Prepare.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/prepare/Prepare.java b/core/src/main/java/org/apache/calcite/prepare/Prepare.java
index 310c219..5f33120 100644
--- a/core/src/main/java/org/apache/calcite/prepare/Prepare.java
+++ b/core/src/main/java/org/apache/calcite/prepare/Prepare.java
@@ -14,36 +14,50 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.prepare;
-
-import net.hydromatic.optiq.DataContext;
-import net.hydromatic.optiq.impl.StarTable;
-import net.hydromatic.optiq.impl.java.JavaTypeFactory;
-import net.hydromatic.optiq.jdbc.OptiqPrepare;
-import net.hydromatic.optiq.jdbc.OptiqSchema;
-import net.hydromatic.optiq.runtime.Bindable;
-import net.hydromatic.optiq.runtime.Hook;
-import net.hydromatic.optiq.runtime.Typed;
-import net.hydromatic.optiq.tools.Program;
-import net.hydromatic.optiq.tools.Programs;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.rex.RexBuilder;
-import org.eigenbase.rex.RexExecutorImpl;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.validate.*;
-import org.eigenbase.sql2rel.SqlToRelConverter;
-import org.eigenbase.trace.EigenbaseTimingTracer;
-import org.eigenbase.trace.EigenbaseTrace;
-import org.eigenbase.util.Holder;
-import org.eigenbase.util.Pair;
+package org.apache.calcite.prepare;
+
+import org.apache.calcite.DataContext;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.jdbc.CalcitePrepare;
+import org.apache.calcite.jdbc.CalciteSchema;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelImplementor;
+import org.apache.calcite.plan.RelOptLattice;
+import org.apache.calcite.plan.RelOptMaterialization;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelOptSchema;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.logical.LogicalTableModify;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexExecutorImpl;
+import org.apache.calcite.runtime.Bindable;
+import org.apache.calcite.runtime.Hook;
+import org.apache.calcite.runtime.Typed;
+import org.apache.calcite.schema.impl.StarTable;
+import org.apache.calcite.sql.SqlExplain;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql.validate.SqlValidatorCatalogReader;
+import org.apache.calcite.sql.validate.SqlValidatorTable;
+import org.apache.calcite.sql2rel.SqlToRelConverter;
+import org.apache.calcite.tools.Program;
+import org.apache.calcite.tools.Programs;
+import org.apache.calcite.util.Holder;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.trace.CalciteTimingTracer;
+import org.apache.calcite.util.trace.CalciteTrace;
 
 import com.google.common.collect.ImmutableList;
 
 import java.lang.reflect.Type;
-import java.util.*;
+import java.util.Collections;
+import java.util.List;
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
@@ -52,16 +66,16 @@ import java.util.logging.Logger;
  * the process of preparing and executing SQL expressions.
  */
 public abstract class Prepare {
-  protected static final Logger LOGGER = EigenbaseTrace.getStatementTracer();
+  protected static final Logger LOGGER = CalciteTrace.getStatementTracer();
 
-  protected final OptiqPrepare.Context context;
+  protected final CalcitePrepare.Context context;
   protected final CatalogReader catalogReader;
   protected String queryString = null;
   /**
    * Convention via which results should be returned by execution.
    */
   protected final Convention resultConvention;
-  protected EigenbaseTimingTracer timingTracer;
+  protected CalciteTimingTracer timingTracer;
   protected List<List<String>> fieldOrigins;
   protected RelDataType parameterRowType;
 
@@ -73,7 +87,7 @@ public abstract class Prepare {
         }
       };
 
-  public Prepare(OptiqPrepare.Context context, CatalogReader catalogReader,
+  public Prepare(CalcitePrepare.Context context, CatalogReader catalogReader,
       Convention resultConvention) {
     assert context != null;
     this.context = context;
@@ -101,7 +115,7 @@ public abstract class Prepare {
    */
   protected RelNode optimize(RelDataType logicalRowType, final RelNode rootRel,
       final List<Materialization> materializations,
-      final List<OptiqSchema.LatticeEntry> lattices) {
+      final List<CalciteSchema.LatticeEntry> lattices) {
     final RelOptPlanner planner = rootRel.getCluster().getPlanner();
 
     planner.setRoot(rootRel);
@@ -119,8 +133,8 @@ public abstract class Prepare {
               materialization.starRelOptTable));
     }
 
-    for (OptiqSchema.LatticeEntry lattice : lattices) {
-      final OptiqSchema.TableEntry starTable = lattice.getStarTable();
+    for (CalciteSchema.LatticeEntry lattice : lattices) {
+      final CalciteSchema.TableEntry starTable = lattice.getStarTable();
       final JavaTypeFactory typeFactory = context.getTypeFactory();
       final RelOptTableImpl starRelOptTable =
           RelOptTableImpl.create(catalogReader,
@@ -174,7 +188,7 @@ public abstract class Prepare {
       SqlValidator validator,
       boolean needsValidation,
       List<Materialization> materializations,
-      List<OptiqSchema.LatticeEntry> lattices) {
+      List<CalciteSchema.LatticeEntry> lattices) {
     return prepareSql(
         sqlQuery,
         sqlQuery,
@@ -192,7 +206,7 @@ public abstract class Prepare {
       SqlValidator validator,
       boolean needsValidation,
       List<Materialization> materializations,
-      List<OptiqSchema.LatticeEntry> lattices) {
+      List<CalciteSchema.LatticeEntry> lattices) {
     queryString = sqlQuery.toString();
 
     init(runtimeContextClass);
@@ -285,20 +299,20 @@ public abstract class Prepare {
         kind);
   }
 
-  protected TableModificationRel.Operation mapTableModOp(
+  protected LogicalTableModify.Operation mapTableModOp(
       boolean isDml, SqlKind sqlKind) {
     if (!isDml) {
       return null;
     }
     switch (sqlKind) {
     case INSERT:
-      return TableModificationRel.Operation.INSERT;
+      return LogicalTableModify.Operation.INSERT;
     case DELETE:
-      return TableModificationRel.Operation.DELETE;
+      return LogicalTableModify.Operation.DELETE;
     case MERGE:
-      return TableModificationRel.Operation.MERGE;
+      return LogicalTableModify.Operation.MERGE;
     case UPDATE:
-      return TableModificationRel.Operation.UPDATE;
+      return LogicalTableModify.Operation.UPDATE;
     default:
       return null;
     }
@@ -329,7 +343,7 @@ public abstract class Prepare {
 
   /**
    * Walks over a tree of relational expressions, replacing each
-   * {@link org.eigenbase.rel.RelNode} with a 'slimmed down' relational
+   * {@link org.apache.calcite.rel.RelNode} with a 'slimmed down' relational
    * expression that projects
    * only the columns required by its consumer.
    *
@@ -429,7 +443,7 @@ public abstract class Prepare {
       return false;
     }
 
-    public TableModificationRel.Operation getTableModOp() {
+    public LogicalTableModify.Operation getTableModOp() {
       return null;
     }
 
@@ -465,7 +479,7 @@ public abstract class Prepare {
      * Returns the table modification operation corresponding to this
      * statement if it is a table modification statement; otherwise null.
      */
-    TableModificationRel.Operation getTableModOp();
+    LogicalTableModify.Operation getTableModOp();
 
     /**
      * Returns a list describing, for each result field, the origin of the
@@ -495,7 +509,7 @@ public abstract class Prepare {
     protected final RelDataType parameterRowType;
     protected final RelDataType rowType;
     protected final boolean isDml;
-    protected final TableModificationRel.Operation tableModOp;
+    protected final LogicalTableModify.Operation tableModOp;
     protected final List<List<String>> fieldOrigins;
 
     public PreparedResultImpl(
@@ -503,7 +517,7 @@ public abstract class Prepare {
         RelDataType parameterRowType,
         List<List<String>> fieldOrigins,
         RelNode rootRel,
-        TableModificationRel.Operation tableModOp,
+        LogicalTableModify.Operation tableModOp,
         boolean isDml) {
       assert rowType != null;
       assert parameterRowType != null;
@@ -521,7 +535,7 @@ public abstract class Prepare {
       return isDml;
     }
 
-    public TableModificationRel.Operation getTableModOp() {
+    public LogicalTableModify.Operation getTableModOp() {
       return tableModOp;
     }
 
@@ -556,18 +570,18 @@ public abstract class Prepare {
    * process. */
   public static class Materialization {
     /** The table that holds the materialized data. */
-    final OptiqSchema.TableEntry materializedTable;
+    final CalciteSchema.TableEntry materializedTable;
     /** The query that derives the data. */
     final String sql;
     /** Relational expression for the table. Usually a
-     * {@link TableAccessRel}. */
+     * {@link org.apache.calcite.rel.logical.LogicalTableScan}. */
     RelNode tableRel;
     /** Relational expression for the query to populate the table. */
     RelNode queryRel;
     /** Star table identified. */
     private RelOptTable starRelOptTable;
 
-    public Materialization(OptiqSchema.TableEntry materializedTable,
+    public Materialization(CalciteSchema.TableEntry materializedTable,
         String sql) {
       assert materializedTable != null;
       assert sql != null;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/prepare/QueryableRelBuilder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/prepare/QueryableRelBuilder.java b/core/src/main/java/org/apache/calcite/prepare/QueryableRelBuilder.java
index d410b92..c2e4d14 100644
--- a/core/src/main/java/org/apache/calcite/prepare/QueryableRelBuilder.java
+++ b/core/src/main/java/org/apache/calcite/prepare/QueryableRelBuilder.java
@@ -14,19 +14,40 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.prepare;
-
-import net.hydromatic.linq4j.*;
-import net.hydromatic.linq4j.expressions.FunctionExpression;
-import net.hydromatic.linq4j.function.*;
-
-import net.hydromatic.optiq.QueryableTable;
-import net.hydromatic.optiq.TranslatableTable;
-import net.hydromatic.optiq.impl.AbstractTableQueryable;
-import net.hydromatic.optiq.jdbc.OptiqSchema;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.rex.RexNode;
+package org.apache.calcite.prepare;
+
+import org.apache.calcite.jdbc.CalciteSchema;
+import org.apache.calcite.linq4j.Enumerable;
+import org.apache.calcite.linq4j.Grouping;
+import org.apache.calcite.linq4j.OrderedQueryable;
+import org.apache.calcite.linq4j.Queryable;
+import org.apache.calcite.linq4j.QueryableDefaults;
+import org.apache.calcite.linq4j.QueryableFactory;
+import org.apache.calcite.linq4j.function.BigDecimalFunction1;
+import org.apache.calcite.linq4j.function.DoubleFunction1;
+import org.apache.calcite.linq4j.function.EqualityComparer;
+import org.apache.calcite.linq4j.function.FloatFunction1;
+import org.apache.calcite.linq4j.function.Function1;
+import org.apache.calcite.linq4j.function.Function2;
+import org.apache.calcite.linq4j.function.IntegerFunction1;
+import org.apache.calcite.linq4j.function.LongFunction1;
+import org.apache.calcite.linq4j.function.NullableBigDecimalFunction1;
+import org.apache.calcite.linq4j.function.NullableDoubleFunction1;
+import org.apache.calcite.linq4j.function.NullableFloatFunction1;
+import org.apache.calcite.linq4j.function.NullableIntegerFunction1;
+import org.apache.calcite.linq4j.function.NullableLongFunction1;
+import org.apache.calcite.linq4j.function.Predicate1;
+import org.apache.calcite.linq4j.function.Predicate2;
+import org.apache.calcite.linq4j.tree.FunctionExpression;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.logical.LogicalFilter;
+import org.apache.calcite.rel.logical.LogicalProject;
+import org.apache.calcite.rel.logical.LogicalTableScan;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.schema.QueryableTable;
+import org.apache.calcite.schema.TranslatableTable;
+import org.apache.calcite.schema.impl.AbstractTableQueryable;
 
 import java.math.BigDecimal;
 import java.util.Comparator;
@@ -42,7 +63,7 @@ import java.util.List;
  * {@link #setRel} to assign the root of that tree to the {@link #rel} member
  * variable.</p>
  *
- * <p>To comply with the {@link net.hydromatic.linq4j.QueryableFactory}
+ * <p>To comply with the {@link org.apache.calcite.linq4j.QueryableFactory}
  * interface, which is after all a factory, each method returns a dummy result
  * such as {@code null} or {@code 0}.
  * The caller will not use the result.
@@ -72,8 +93,8 @@ class QueryableRelBuilder<T> implements QueryableFactory<T> {
       final AbstractTableQueryable tableQueryable =
           (AbstractTableQueryable) queryable;
       final QueryableTable table = tableQueryable.table;
-      final OptiqSchema.TableEntry tableEntry =
-          OptiqSchema.from(tableQueryable.schema)
+      final CalciteSchema.TableEntry tableEntry =
+          CalciteSchema.from(tableQueryable.schema)
               .add(tableQueryable.tableName, tableQueryable.table);
       final RelOptTableImpl relOptTable =
           RelOptTableImpl.create(null, table.getRowType(translator.typeFactory),
@@ -81,7 +102,7 @@ class QueryableRelBuilder<T> implements QueryableFactory<T> {
       if (table instanceof TranslatableTable) {
         return ((TranslatableTable) table).toRel(translator, relOptTable);
       } else {
-        return new TableAccessRel(translator.cluster, relOptTable);
+        return new LogicalTableScan(translator.cluster, relOptTable);
       }
     }
     return translator.translate(queryable.getExpression());
@@ -490,12 +511,12 @@ class QueryableRelBuilder<T> implements QueryableFactory<T> {
     RelNode child = toRel(source);
     List<RexNode> nodes = translator.toRexList(selector, child);
     setRel(
-        new ProjectRel(
+        new LogicalProject(
             translator.cluster,
             child,
             nodes,
             null,
-            ProjectRelBase.Flags.BOXED));
+            Project.Flags.BOXED));
     return null;
   }
 
@@ -702,7 +723,7 @@ class QueryableRelBuilder<T> implements QueryableFactory<T> {
       FunctionExpression<? extends Predicate1<T>> predicate) {
     RelNode child = toRel(source);
     RexNode node = translator.toRex(predicate, child);
-    setRel(new FilterRel(translator.cluster, child, node));
+    setRel(new LogicalFilter(translator.cluster, child, node));
     return source;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/prepare/RelOptTableImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/prepare/RelOptTableImpl.java b/core/src/main/java/org/apache/calcite/prepare/RelOptTableImpl.java
index 21b3dbf..a3377a0 100644
--- a/core/src/main/java/org/apache/calcite/prepare/RelOptTableImpl.java
+++ b/core/src/main/java/org/apache/calcite/prepare/RelOptTableImpl.java
@@ -14,23 +14,28 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.prepare;
-
-import net.hydromatic.linq4j.expressions.Expression;
-
-import net.hydromatic.optiq.*;
-import net.hydromatic.optiq.jdbc.OptiqSchema;
-import net.hydromatic.optiq.rules.java.EnumerableConvention;
-import net.hydromatic.optiq.rules.java.JavaRules;
-
-import org.eigenbase.rel.RelCollation;
-import org.eigenbase.rel.RelNode;
-import org.eigenbase.relopt.RelOptCluster;
-import org.eigenbase.relopt.RelOptSchema;
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.sql.SqlAccessType;
-import org.eigenbase.sql.validate.SqlMonotonicity;
-import org.eigenbase.util.Util;
+package org.apache.calcite.prepare;
+
+import org.apache.calcite.adapter.enumerable.EnumerableConvention;
+import org.apache.calcite.adapter.enumerable.EnumerableInterpreter;
+import org.apache.calcite.adapter.enumerable.EnumerableTableScan;
+import org.apache.calcite.jdbc.CalciteSchema;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptSchema;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.schema.FilterableTable;
+import org.apache.calcite.schema.ProjectableFilterableTable;
+import org.apache.calcite.schema.QueryableTable;
+import org.apache.calcite.schema.ScannableTable;
+import org.apache.calcite.schema.Schemas;
+import org.apache.calcite.schema.Table;
+import org.apache.calcite.schema.TranslatableTable;
+import org.apache.calcite.sql.SqlAccessType;
+import org.apache.calcite.sql.validate.SqlMonotonicity;
+import org.apache.calcite.util.Util;
 
 import com.google.common.base.Function;
 import com.google.common.base.Functions;
@@ -42,7 +47,7 @@ import java.util.Collections;
 import java.util.List;
 
 /**
- * Implementation of {@link org.eigenbase.relopt.RelOptTable}.
+ * Implementation of {@link org.apache.calcite.plan.RelOptTable}.
  */
 public class RelOptTableImpl implements Prepare.PreparingTable {
   private final RelOptSchema schema;
@@ -90,7 +95,7 @@ public class RelOptTableImpl implements Prepare.PreparingTable {
   }
 
   public static RelOptTableImpl create(RelOptSchema schema, RelDataType rowType,
-      final OptiqSchema.TableEntry tableEntry, Double rowCount) {
+      final CalciteSchema.TableEntry tableEntry, Double rowCount) {
     Function<Class, Expression> expressionFunction;
     final Table table = tableEntry.getTable();
     if (table instanceof QueryableTable) {
@@ -143,9 +148,9 @@ public class RelOptTableImpl implements Prepare.PreparingTable {
     if (clazz.isInstance(table)) {
       return clazz.cast(table);
     }
-    if (clazz == OptiqSchema.class) {
+    if (clazz == CalciteSchema.class) {
       return clazz.cast(
-          Schemas.subSchema(((OptiqCatalogReader) schema).rootSchema,
+          Schemas.subSchema(((CalciteCatalogReader) schema).rootSchema,
               Util.skipLast(getQualifiedName())));
     }
     return null;
@@ -178,11 +183,11 @@ public class RelOptTableImpl implements Prepare.PreparingTable {
     }
     RelOptCluster cluster = context.getCluster();
     Class elementType = deduceElementType();
-    final RelNode scan = new JavaRules.EnumerableTableAccessRel(cluster,
+    final RelNode scan = new EnumerableTableScan(cluster,
         cluster.traitSetOf(EnumerableConvention.INSTANCE), this, elementType);
     if (table instanceof FilterableTable
         || table instanceof ProjectableFilterableTable) {
-      return new JavaRules.EnumerableInterpreterRel(cluster, scan.getTraitSet(),
+      return new EnumerableInterpreter(cluster, scan.getTraitSet(),
           scan, 1d);
     }
     return scan;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/prepare/package-info.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/prepare/package-info.java b/core/src/main/java/org/apache/calcite/prepare/package-info.java
index 88efbe3..0d4dd6e 100644
--- a/core/src/main/java/org/apache/calcite/prepare/package-info.java
+++ b/core/src/main/java/org/apache/calcite/prepare/package-info.java
@@ -18,6 +18,6 @@
 /**
  * Preparation of queries (parsing, planning and implementation).
  */
-package net.hydromatic.optiq.prepare;
+package org.apache.calcite.prepare;
 
 // End package-info.java


[46/58] [abbrv] [partial] incubator-calcite git commit: [CALCITE-306] Standardize code style for "import package.*; "

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableWindow.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableWindow.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableWindow.java
new file mode 100644
index 0000000..de0ddd1
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableWindow.java
@@ -0,0 +1,952 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.adapter.enumerable.impl.WinAggAddContextImpl;
+import org.apache.calcite.adapter.enumerable.impl.WinAggResetContextImpl;
+import org.apache.calcite.adapter.enumerable.impl.WinAggResultContextImpl;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.linq4j.tree.BinaryExpression;
+import org.apache.calcite.linq4j.tree.BlockBuilder;
+import org.apache.calcite.linq4j.tree.BlockStatement;
+import org.apache.calcite.linq4j.tree.DeclarationStatement;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.linq4j.tree.ParameterExpression;
+import org.apache.calcite.linq4j.tree.Primitive;
+import org.apache.calcite.linq4j.tree.Statement;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.prepare.CalcitePrepareImpl;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.core.Window;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexWindowBound;
+import org.apache.calcite.runtime.SortedMultiMap;
+import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.BuiltInMethod;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
+
+import com.google.common.base.Function;
+import com.google.common.collect.ImmutableList;
+
+import java.lang.reflect.Modifier;
+import java.lang.reflect.Type;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
+
+/** Implementation of {@link org.apache.calcite.rel.core.Window} in
+ * {@link org.apache.calcite.adapter.enumerable.EnumerableConvention enumerable calling convention}. */
+public class EnumerableWindow extends Window implements EnumerableRel {
+  /** Creates an EnumerableWindowRel. */
+  EnumerableWindow(RelOptCluster cluster, RelTraitSet traits, RelNode child,
+      List<RexLiteral> constants, RelDataType rowType, List<Group> groups) {
+    super(cluster, traits, child, constants, rowType, groups);
+  }
+
+  @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+    return new EnumerableWindow(getCluster(), traitSet, sole(inputs),
+        constants, rowType, groups);
+  }
+
+  public RelOptCost computeSelfCost(RelOptPlanner planner) {
+    // Cost is proportional to the number of rows and the number of
+    // components (groups and aggregate functions). There is
+    // no I/O cost.
+    //
+    // TODO #1. Add memory cost.
+    // TODO #2. MIN and MAX have higher CPU cost than SUM and COUNT.
+    final double rowsIn = RelMetadataQuery.getRowCount(getInput());
+    int count = groups.size();
+    for (Group group : groups) {
+      count += group.aggCalls.size();
+    }
+    return planner.getCostFactory().makeCost(rowsIn, rowsIn * count, 0);
+  }
+
+  /** Implementation of {@link RexToLixTranslator.InputGetter}
+   * suitable for generating implementations of windowed aggregate
+   * functions. */
+  private static class WindowRelInputGetter
+      implements RexToLixTranslator.InputGetter {
+    private final Expression row;
+    private final PhysType rowPhysType;
+    private final int actualInputFieldCount;
+    private final List<Expression> constants;
+
+    private WindowRelInputGetter(Expression row,
+        PhysType rowPhysType, int actualInputFieldCount,
+        List<Expression> constants) {
+      this.row = row;
+      this.rowPhysType = rowPhysType;
+      this.actualInputFieldCount = actualInputFieldCount;
+      this.constants = constants;
+    }
+
+    public Expression field(BlockBuilder list, int index, Type storageType) {
+      if (index < actualInputFieldCount) {
+        Expression current = list.append("current", row);
+        return rowPhysType.fieldReference(current, index, storageType);
+      }
+      return constants.get(index - actualInputFieldCount);
+    }
+  }
+
+  private void sampleOfTheGeneratedWindowedAggregate() {
+    // Here's overview of the generated code
+    // For each list of rows that have the same partitioning key, evaluate
+    // all of the windowed aggregate functions.
+
+    // builder
+    Iterator<Integer[]> iterator = null;
+
+    // builder3
+    Integer[] rows = iterator.next();
+
+    int prevStart = -1;
+    int prevEnd = -1;
+
+    for (int i = 0; i < rows.length; i++) {
+      // builder4
+      Integer row = rows[i];
+
+      int start = 0;
+      int end = 100;
+      if (start != prevStart || end != prevEnd) {
+        // builder5
+        int actualStart = 0;
+        if (start != prevStart || end < prevEnd) {
+          // builder6
+          // recompute
+          actualStart = start;
+          // implementReset
+        } else { // must be start == prevStart && end > prevEnd
+          actualStart = prevEnd + 1;
+        }
+        prevStart = start;
+        prevEnd = end;
+
+        if (start != -1) {
+          for (int j = actualStart; j <= end; j++) {
+            // builder7
+            // implementAdd
+          }
+        }
+        // implementResult
+        // list.add(new Xxx(row.deptno, row.empid, sum, count));
+      }
+    }
+    // multiMap.clear(); // allows gc
+    // source = Linq4j.asEnumerable(list);
+  }
+
+  public Result implement(EnumerableRelImplementor implementor, Prefer pref) {
+    final JavaTypeFactory typeFactory = implementor.getTypeFactory();
+    final EnumerableRel child = (EnumerableRel) getInput();
+    final BlockBuilder builder = new BlockBuilder();
+    final Result result = implementor.visitChild(this, 0, child, pref);
+    Expression source_ = builder.append("source", result.block);
+
+    final List<Expression> translatedConstants =
+        new ArrayList<Expression>(constants.size());
+    for (RexLiteral constant : constants) {
+      translatedConstants.add(RexToLixTranslator.translateLiteral(
+          constant, constant.getType(),
+          typeFactory,
+          RexImpTable.NullAs.NULL));
+    }
+
+    PhysType inputPhysType = result.physType;
+
+    ParameterExpression prevStart =
+        Expressions.parameter(int.class, builder.newName("prevStart"));
+    ParameterExpression prevEnd =
+        Expressions.parameter(int.class, builder.newName("prevEnd"));
+
+    builder.add(Expressions.declare(0, prevStart, null));
+    builder.add(Expressions.declare(0, prevEnd, null));
+
+    for (int windowIdx = 0; windowIdx < groups.size(); windowIdx++) {
+      Group group = groups.get(windowIdx);
+      // Comparator:
+      // final Comparator<JdbcTest.Employee> comparator =
+      //    new Comparator<JdbcTest.Employee>() {
+      //      public int compare(JdbcTest.Employee o1,
+      //          JdbcTest.Employee o2) {
+      //        return Integer.compare(o1.empid, o2.empid);
+      //      }
+      //    };
+      final Expression comparator_ =
+          builder.append(
+              "comparator",
+              inputPhysType.generateComparator(
+                  group.collation()));
+
+      Pair<Expression, Expression> partitionIterator =
+          getPartitionIterator(builder, source_, inputPhysType, group,
+              comparator_);
+      final Expression collectionExpr = partitionIterator.left;
+      final Expression iterator_ = partitionIterator.right;
+
+      List<AggImpState> aggs = new ArrayList<AggImpState>();
+      List<AggregateCall> aggregateCalls = group.getAggregateCalls(this);
+      for (int aggIdx = 0; aggIdx < aggregateCalls.size(); aggIdx++) {
+        AggregateCall call = aggregateCalls.get(aggIdx);
+        aggs.add(new AggImpState(aggIdx, call, true));
+      }
+
+      // The output from this stage is the input plus the aggregate functions.
+      final RelDataTypeFactory.FieldInfoBuilder typeBuilder =
+          typeFactory.builder();
+      typeBuilder.addAll(inputPhysType.getRowType().getFieldList());
+      for (AggImpState agg : aggs) {
+        typeBuilder.add(agg.call.name, agg.call.type);
+      }
+      RelDataType outputRowType = typeBuilder.build();
+      final PhysType outputPhysType =
+          PhysTypeImpl.of(
+              typeFactory, outputRowType, pref.prefer(result.format));
+
+      final Expression list_ =
+          builder.append(
+              "list",
+              Expressions.new_(
+                  ArrayList.class,
+                  Expressions.call(
+                      collectionExpr, BuiltInMethod.COLLECTION_SIZE.method)),
+              false);
+
+      Pair<Expression, Expression> collationKey =
+          getRowCollationKey(builder, inputPhysType, group, windowIdx);
+      Expression keySelector = collationKey.left;
+      Expression keyComparator = collationKey.right;
+      final BlockBuilder builder3 = new BlockBuilder();
+      final Expression rows_ =
+          builder3.append(
+              "rows",
+              Expressions.convert_(
+                  Expressions.call(
+                      iterator_, BuiltInMethod.ITERATOR_NEXT.method),
+                  Object[].class),
+              false);
+
+      builder3.add(Expressions.statement(
+          Expressions.assign(prevStart, Expressions.constant(-1))));
+      builder3.add(Expressions.statement(
+          Expressions.assign(prevEnd,
+              Expressions.constant(Integer.MAX_VALUE))));
+
+      final BlockBuilder builder4 = new BlockBuilder();
+
+      final ParameterExpression i_ =
+          Expressions.parameter(int.class, builder4.newName("i"));
+
+      final Expression row_ =
+          builder4.append(
+              "row",
+              RexToLixTranslator.convert(
+                  Expressions.arrayIndex(rows_, i_),
+                  inputPhysType.getJavaRowType()));
+
+      final RexToLixTranslator.InputGetter inputGetter =
+          new WindowRelInputGetter(row_, inputPhysType,
+              result.physType.getRowType().getFieldCount(),
+              translatedConstants);
+
+      final RexToLixTranslator translator =
+          RexToLixTranslator.forAggregation(typeFactory, builder4,
+              inputGetter);
+
+      final List<Expression> outputRow = new ArrayList<Expression>();
+      int fieldCountWithAggResults =
+        inputPhysType.getRowType().getFieldCount();
+      for (int i = 0; i < fieldCountWithAggResults; i++) {
+        outputRow.add(
+            inputPhysType.fieldReference(
+                row_, i,
+                outputPhysType.getJavaFieldType(i)));
+      }
+
+      declareAndResetState(typeFactory, builder, result, windowIdx, aggs,
+          outputPhysType, outputRow);
+
+      // There are assumptions that minX==0. If ever change this, look for
+      // frameRowCount, bounds checking, etc
+      final Expression minX = Expressions.constant(0);
+      final Expression partitionRowCount =
+          builder3.append("partRows", Expressions.field(rows_, "length"));
+      final Expression maxX = builder3.append("maxX",
+          Expressions.subtract(
+              partitionRowCount, Expressions.constant(1)));
+
+      final Expression startUnchecked = builder4.append("start",
+          translateBound(translator, i_, row_, minX, maxX, rows_,
+              group, true,
+              inputPhysType, comparator_, keySelector, keyComparator));
+      final Expression endUnchecked = builder4.append("end",
+          translateBound(translator, i_, row_, minX, maxX, rows_,
+              group, false,
+              inputPhysType, comparator_, keySelector, keyComparator));
+
+      final Expression startX;
+      final Expression endX;
+      final Expression hasRows;
+      if (group.isAlwaysNonEmpty()) {
+        startX = startUnchecked;
+        endX = endUnchecked;
+        hasRows = Expressions.constant(true);
+      } else {
+        Expression startTmp =
+            group.lowerBound.isUnbounded() || startUnchecked == i_
+                ? startUnchecked
+                : builder4.append("startTmp",
+                    Expressions.call(null, BuiltInMethod.MATH_MAX.method,
+                        startUnchecked, minX));
+        Expression endTmp =
+            group.upperBound.isUnbounded() || endUnchecked == i_
+                ? endUnchecked
+                : builder4.append("endTmp",
+                    Expressions.call(null, BuiltInMethod.MATH_MIN.method,
+                        endUnchecked, maxX));
+
+        ParameterExpression startPe = Expressions.parameter(0, int.class,
+            builder4.newName("startChecked"));
+        ParameterExpression endPe = Expressions.parameter(0, int.class,
+            builder4.newName("endChecked"));
+        builder4.add(Expressions.declare(Modifier.FINAL, startPe, null));
+        builder4.add(Expressions.declare(Modifier.FINAL, endPe, null));
+
+        hasRows = builder4.append("hasRows",
+            Expressions.lessThanOrEqual(startTmp, endTmp));
+        builder4.add(Expressions.ifThenElse(
+            hasRows,
+            Expressions.block(
+                Expressions.statement(
+                    Expressions.assign(startPe, startTmp)),
+                Expressions.statement(
+                  Expressions.assign(endPe, endTmp))
+          ),
+            Expressions.block(
+                Expressions.statement(
+                    Expressions.assign(startPe, Expressions.constant(-1))),
+                Expressions.statement(
+                    Expressions.assign(endPe, Expressions.constant(-1))))));
+        startX = startPe;
+        endX = endPe;
+      }
+
+      final BlockBuilder builder5 = new BlockBuilder(true, builder4);
+
+      BinaryExpression rowCountWhenNonEmpty = Expressions.add(
+          startX == minX ? endX : Expressions.subtract(endX, startX),
+          Expressions.constant(1));
+
+      final Expression frameRowCount;
+
+      if (hasRows.equals(Expressions.constant(true))) {
+        frameRowCount =
+            builder4.append("totalRows", rowCountWhenNonEmpty);
+      } else {
+        frameRowCount =
+            builder4.append("totalRows", Expressions.condition(hasRows,
+                rowCountWhenNonEmpty, Expressions.constant(0)));
+      }
+
+      ParameterExpression actualStart = Expressions.parameter(
+          0, int.class, builder5.newName("actualStart"));
+
+      final BlockBuilder builder6 = new BlockBuilder(true, builder5);
+      builder6.add(Expressions.statement(
+          Expressions.assign(actualStart, startX)));
+
+      for (final AggImpState agg : aggs) {
+        agg.implementor.implementReset(agg.context,
+            new WinAggResetContextImpl(builder6, agg.state, i_, startX, endX,
+                hasRows, partitionRowCount, frameRowCount));
+      }
+
+      Expression lowerBoundCanChange =
+          group.lowerBound.isUnbounded() && group.lowerBound.isPreceding()
+          ? Expressions.constant(false)
+          : Expressions.notEqual(startX, prevStart);
+      Expression needRecomputeWindow = Expressions.orElse(
+          lowerBoundCanChange,
+          Expressions.lessThan(endX, prevEnd));
+
+      BlockStatement resetWindowState = builder6.toBlock();
+      if (resetWindowState.statements.size() == 1) {
+        builder5.add(Expressions.declare(0, actualStart,
+            Expressions.condition(needRecomputeWindow,
+                startX, Expressions.add(prevEnd, Expressions.constant(1)))));
+      } else {
+        builder5.add(Expressions.declare(0, actualStart,
+            null));
+        builder5.add(Expressions.ifThenElse(needRecomputeWindow,
+            resetWindowState,
+            Expressions.statement(Expressions.assign(actualStart,
+                Expressions.add(prevEnd, Expressions.constant(1))))));
+      }
+
+      if (lowerBoundCanChange instanceof BinaryExpression) {
+        builder5.add(Expressions.statement(
+            Expressions.assign(prevStart, startX)));
+      }
+      builder5.add(Expressions.statement(
+          Expressions.assign(prevEnd, endX)));
+
+      final BlockBuilder builder7 = new BlockBuilder(true, builder5);
+      final DeclarationStatement jDecl =
+          Expressions.declare(0, "j", actualStart);
+
+      final PhysType inputPhysTypeFinal = inputPhysType;
+      final Function<BlockBuilder, WinAggFrameResultContext>
+          resultContextBuilder =
+          getBlockBuilderWinAggFrameResultContextFunction(typeFactory, result,
+              translatedConstants, comparator_, rows_, i_, startX, endX,
+              minX, maxX,
+              hasRows, frameRowCount, partitionRowCount,
+              jDecl, inputPhysTypeFinal);
+
+      final Function<AggImpState, List<RexNode>> rexArguments =
+          new Function<AggImpState, List<RexNode>>() {
+            public List<RexNode> apply(AggImpState agg) {
+              List<Integer> argList = agg.call.getArgList();
+              List<RelDataType> inputTypes =
+                  EnumUtils.fieldRowTypes(
+                      result.physType.getRowType(),
+                      constants,
+                      argList);
+              List<RexNode> args = new ArrayList<RexNode>(
+                  inputTypes.size());
+              for (int i = 0; i < argList.size(); i++) {
+                Integer idx = argList.get(i);
+                args.add(new RexInputRef(idx, inputTypes.get(i)));
+              }
+              return args;
+            }
+          };
+
+      implementAdd(aggs, builder7, resultContextBuilder, rexArguments, jDecl);
+
+      BlockStatement forBlock = builder7.toBlock();
+      if (!forBlock.statements.isEmpty()) {
+        // For instance, row_number does not use for loop to compute the value
+        Statement forAggLoop = Expressions.for_(
+            Arrays.asList(jDecl),
+            Expressions.lessThanOrEqual(jDecl.parameter, endX),
+            Expressions.preIncrementAssign(jDecl.parameter),
+            forBlock);
+        if (!hasRows.equals(Expressions.constant(true))) {
+          forAggLoop = Expressions.ifThen(hasRows, forAggLoop);
+        }
+        builder5.add(forAggLoop);
+      }
+
+      if (implementResult(aggs, builder5, resultContextBuilder, rexArguments,
+              true)) {
+        builder4.add(Expressions.ifThen(Expressions.orElse(
+            lowerBoundCanChange,
+            Expressions.notEqual(endX, prevEnd)), builder5.toBlock()));
+      }
+
+      implementResult(aggs, builder4, resultContextBuilder, rexArguments,
+          false);
+
+      builder4.add(
+          Expressions.statement(
+              Expressions.call(
+                  list_,
+                  BuiltInMethod.COLLECTION_ADD.method,
+                  outputPhysType.record(outputRow))));
+
+      builder3.add(
+          Expressions.for_(
+              Expressions.declare(0, i_, Expressions.constant(0)),
+              Expressions.lessThan(
+                  i_,
+                  Expressions.field(rows_, "length")),
+              Expressions.preIncrementAssign(i_),
+              builder4.toBlock()));
+
+      builder.add(
+          Expressions.while_(
+              Expressions.call(
+                  iterator_,
+                  BuiltInMethod.ITERATOR_HAS_NEXT.method),
+              builder3.toBlock()));
+      builder.add(
+          Expressions.statement(
+              Expressions.call(
+                  collectionExpr,
+                  BuiltInMethod.MAP_CLEAR.method)));
+
+      // We're not assigning to "source". For each group, create a new
+      // final variable called "source" or "sourceN".
+      source_ =
+          builder.append(
+              "source",
+              Expressions.call(
+                  BuiltInMethod.AS_ENUMERABLE.method, list_));
+
+      inputPhysType = outputPhysType;
+    }
+
+    //   return Linq4j.asEnumerable(list);
+    builder.add(
+        Expressions.return_(null, source_));
+    return implementor.result(inputPhysType, builder.toBlock());
+  }
+
+  private Function<BlockBuilder, WinAggFrameResultContext>
+  getBlockBuilderWinAggFrameResultContextFunction(
+      final JavaTypeFactory typeFactory, final Result result,
+      final List<Expression> translatedConstants,
+      final Expression comparator_,
+      final Expression rows_, final ParameterExpression i_,
+      final Expression startX, final Expression endX,
+      final Expression minX, final Expression maxX,
+      final Expression hasRows, final Expression frameRowCount,
+      final Expression partitionRowCount,
+      final DeclarationStatement jDecl,
+      final PhysType inputPhysType) {
+    return new Function<BlockBuilder,
+        WinAggFrameResultContext>() {
+      public WinAggFrameResultContext apply(
+          final BlockBuilder block) {
+        return new WinAggFrameResultContext() {
+          public RexToLixTranslator rowTranslator(Expression rowIndex) {
+            Expression row =
+                getRow(rowIndex);
+            final RexToLixTranslator.InputGetter inputGetter =
+                new WindowRelInputGetter(row, inputPhysType,
+                    result.physType.getRowType().getFieldCount(),
+                    translatedConstants);
+
+            return RexToLixTranslator.forAggregation(typeFactory,
+                block, inputGetter);
+          }
+
+          public Expression computeIndex(Expression offset,
+              WinAggImplementor.SeekType seekType) {
+            Expression index;
+            if (seekType == WinAggImplementor.SeekType.AGG_INDEX) {
+              index = jDecl.parameter;
+            } else if (seekType == WinAggImplementor.SeekType.SET) {
+              index = i_;
+            } else if (seekType == WinAggImplementor.SeekType.START) {
+              index = startX;
+            } else if (seekType == WinAggImplementor.SeekType.END) {
+              index = endX;
+            } else {
+              throw new IllegalArgumentException("SeekSet " + seekType
+                  + " is not supported");
+            }
+            if (!Expressions.constant(0).equals(offset)) {
+              index = block.append("idx", Expressions.add(index, offset));
+            }
+            return index;
+          }
+
+          private Expression checkBounds(Expression rowIndex,
+              Expression minIndex, Expression maxIndex) {
+            if (rowIndex == i_ || rowIndex == startX || rowIndex == endX) {
+              // No additional bounds check required
+              return hasRows;
+            }
+
+            //noinspection UnnecessaryLocalVariable
+            Expression res = block.append("rowInFrame", Expressions.foldAnd(
+                ImmutableList.of(hasRows,
+                    Expressions.greaterThanOrEqual(rowIndex, minIndex),
+                    Expressions.lessThanOrEqual(rowIndex, maxIndex))));
+
+            return res;
+          }
+
+          public Expression rowInFrame(Expression rowIndex) {
+            return checkBounds(rowIndex, startX, endX);
+          }
+
+          public Expression rowInPartition(Expression rowIndex) {
+            return checkBounds(rowIndex, minX, maxX);
+          }
+
+          public Expression compareRows(Expression a, Expression b) {
+            return Expressions.call(comparator_,
+                BuiltInMethod.COMPARATOR_COMPARE.method,
+                getRow(a), getRow(b));
+          }
+
+          public Expression getRow(Expression rowIndex) {
+            return block.append(
+                "jRow",
+                RexToLixTranslator.convert(
+                    Expressions.arrayIndex(rows_, rowIndex),
+                    inputPhysType.getJavaRowType()));
+          }
+
+          public Expression index() {
+            return i_;
+          }
+
+          public Expression startIndex() {
+            return startX;
+          }
+
+          public Expression endIndex() {
+            return endX;
+          }
+
+          public Expression hasRows() {
+            return hasRows;
+          }
+
+          public Expression getFrameRowCount() {
+            return frameRowCount;
+          }
+
+          public Expression getPartitionRowCount() {
+            return partitionRowCount;
+          }
+        };
+      }
+    };
+  }
+
+  private Pair<Expression, Expression> getPartitionIterator(
+      BlockBuilder builder,
+      Expression source_,
+      PhysType inputPhysType,
+      Group group,
+      Expression comparator_) {
+    // Populate map of lists, one per partition
+    //   final Map<Integer, List<Employee>> multiMap =
+    //     new SortedMultiMap<Integer, List<Employee>>();
+    //    source.foreach(
+    //      new Function1<Employee, Void>() {
+    //        public Void apply(Employee v) {
+    //          final Integer k = v.deptno;
+    //          multiMap.putMulti(k, v);
+    //          return null;
+    //        }
+    //      });
+    //   final List<Xxx> list = new ArrayList<Xxx>(multiMap.size());
+    //   Iterator<Employee[]> iterator = multiMap.arrays(comparator);
+    //
+    if (group.keys.isEmpty()) {
+      // If partition key is empty, no need to partition.
+      //
+      //   final List<Employee> tempList =
+      //       source.into(new ArrayList<Employee>());
+      //   Iterator<Employee[]> iterator =
+      //       SortedMultiMap.singletonArrayIterator(comparator, tempList);
+      //   final List<Xxx> list = new ArrayList<Xxx>(tempList.size());
+
+      final Expression tempList_ = builder.append(
+          "tempList",
+          Expressions.convert_(
+              Expressions.call(
+                  source_,
+                  BuiltInMethod.INTO.method,
+                  Expressions.new_(ArrayList.class)),
+              List.class));
+      return Pair.of(tempList_,
+          builder.append(
+            "iterator",
+            Expressions.call(
+                null,
+                BuiltInMethod.SORTED_MULTI_MAP_SINGLETON.method,
+                comparator_,
+                tempList_)));
+    }
+    Expression multiMap_ =
+        builder.append(
+            "multiMap", Expressions.new_(SortedMultiMap.class));
+    final BlockBuilder builder2 = new BlockBuilder();
+    final ParameterExpression v_ =
+        Expressions.parameter(inputPhysType.getJavaRowType(),
+            builder2.newName("v"));
+    final DeclarationStatement declare =
+        Expressions.declare(
+            0, "key",
+            inputPhysType.selector(
+                v_,
+                BitSets.toList(group.keys),
+                JavaRowFormat.CUSTOM));
+    builder2.add(declare);
+    final ParameterExpression key_ = declare.parameter;
+    builder2.add(
+        Expressions.statement(
+            Expressions.call(
+                multiMap_,
+                BuiltInMethod.SORTED_MULTI_MAP_PUT_MULTI.method,
+                key_,
+                v_)));
+    builder2.add(
+        Expressions.return_(
+            null, Expressions.constant(null)));
+
+    builder.add(
+        Expressions.statement(
+            Expressions.call(
+                source_,
+                BuiltInMethod.ENUMERABLE_FOREACH.method,
+                Expressions.lambda(
+                    builder2.toBlock(), v_))));
+
+    return Pair.of(multiMap_,
+      builder.append(
+        "iterator",
+        Expressions.call(
+            multiMap_,
+            BuiltInMethod.SORTED_MULTI_MAP_ARRAYS.method,
+            comparator_)));
+  }
+
+  private Pair<Expression, Expression> getRowCollationKey(
+      BlockBuilder builder, PhysType inputPhysType,
+      Group group, int windowIdx) {
+    if (!(group.isRows || (group.upperBound.isUnbounded()
+        && group.lowerBound.isUnbounded()))) {
+      Pair<Expression, Expression> pair =
+          inputPhysType.generateCollationKey(
+              group.collation().getFieldCollations());
+      // optimize=false to prevent inlining of object create into for-loops
+      return Pair.of(
+          builder.append("keySelector" + windowIdx, pair.left, false),
+          builder.append("keyComparator" + windowIdx, pair.right, false));
+    } else {
+      return Pair.of(null, null);
+    }
+  }
+
+  private void declareAndResetState(final JavaTypeFactory typeFactory,
+      BlockBuilder builder, final Result result, int windowIdx,
+      List<AggImpState> aggs, PhysType outputPhysType,
+      List<Expression> outputRow) {
+    for (final AggImpState agg: aggs) {
+      agg.context =
+          new WinAggContext() {
+            public org.apache.calcite.sql.SqlAggFunction aggregation() {
+              return agg.call.getAggregation();
+            }
+
+            public RelDataType returnRelType() {
+              return agg.call.type;
+            }
+
+            public Type returnType() {
+              return EnumUtils.javaClass(typeFactory, returnRelType());
+            }
+
+            public List<? extends Type> parameterTypes() {
+              return EnumUtils.fieldTypes(typeFactory,
+                  parameterRelTypes());
+            }
+
+            public List<? extends RelDataType> parameterRelTypes() {
+              return EnumUtils.fieldRowTypes(result.physType.getRowType(),
+                  constants, agg.call.getArgList());
+            }
+          };
+      String aggName = "a" + agg.aggIdx;
+      if (CalcitePrepareImpl.DEBUG) {
+        aggName = Util.toJavaId(agg.call.getAggregation().getName(), 0)
+            .substring("ID$0$".length()) + aggName;
+      }
+      List<Type> state = agg.implementor.getStateType(agg.context);
+      final List<Expression> decls =
+          new ArrayList<Expression>(state.size());
+      for (int i = 0; i < state.size(); i++) {
+        Type type = state.get(i);
+        ParameterExpression pe =
+            Expressions.parameter(type,
+                builder.newName(aggName
+                    + "s" + i + "w" + windowIdx));
+        builder.add(Expressions.declare(0, pe, null));
+        decls.add(pe);
+      }
+      agg.state = decls;
+      Type aggHolderType = agg.context.returnType();
+      Type aggStorageType =
+          outputPhysType.getJavaFieldType(outputRow.size());
+      if (Primitive.is(aggHolderType) && !Primitive.is(aggStorageType)) {
+        aggHolderType = Primitive.box(aggHolderType);
+      }
+      ParameterExpression aggRes = Expressions.parameter(0,
+          aggHolderType,
+          builder.newName(aggName + "w" + windowIdx));
+
+      builder.add(Expressions.declare(0, aggRes,
+          Expressions.constant(
+              Primitive.is(aggRes.getType())
+                  ? Primitive.of(aggRes.getType()).defaultValue
+                  : null, aggRes.getType())));
+      agg.result = aggRes;
+      outputRow.add(aggRes);
+      agg.implementor.implementReset(agg.context,
+          new WinAggResetContextImpl(builder, agg.state,
+              null, null, null, null, null, null));
+    }
+  }
+
+  private void implementAdd(List<AggImpState> aggs,
+      final BlockBuilder builder7,
+      final Function<BlockBuilder, WinAggFrameResultContext> frame,
+      final Function<AggImpState, List<RexNode>> rexArguments,
+      final DeclarationStatement jDecl) {
+    for (final AggImpState agg : aggs) {
+      final WinAggAddContext addContext =
+          new WinAggAddContextImpl(builder7, agg.state, frame) {
+            public Expression currentPosition() {
+              return jDecl.parameter;
+            }
+
+            public List<RexNode> rexArguments() {
+              return rexArguments.apply(agg);
+            }
+          };
+      agg.implementor.implementAdd(agg.context, addContext);
+    }
+  }
+
+  private boolean implementResult(List<AggImpState> aggs,
+      final BlockBuilder builder,
+      final Function<BlockBuilder, WinAggFrameResultContext> frame,
+      final Function<AggImpState, List<RexNode>> rexArguments,
+      boolean cachedBlock) {
+    boolean nonEmpty = false;
+    for (final AggImpState agg : aggs) {
+      boolean needCache = true;
+      if (agg.implementor instanceof WinAggImplementor) {
+        WinAggImplementor imp = (WinAggImplementor) agg.implementor;
+        needCache = imp.needCacheWhenFrameIntact();
+      }
+      if (needCache ^ cachedBlock) {
+        // Regular aggregates do not change when the windowing frame keeps
+        // the same. Ths
+        continue;
+      }
+      nonEmpty = true;
+      Expression res = agg.implementor.implementResult(agg.context,
+          new WinAggResultContextImpl(builder, agg.state, frame) {
+            public List<RexNode> rexArguments() {
+              return rexArguments.apply(agg);
+            }
+          });
+      // Several count(a) and count(b) might share the result
+      Expression aggRes = builder.append("a" + agg.aggIdx + "res",
+          RexToLixTranslator.convert(res, agg.result.getType()));
+      builder.add(Expressions.statement(
+          Expressions.assign(agg.result, aggRes)));
+    }
+    return nonEmpty;
+  }
+
+  private Expression translateBound(RexToLixTranslator translator,
+      ParameterExpression i_, Expression row_, Expression min_,
+      Expression max_, Expression rows_, Group group,
+      boolean lower,
+      PhysType physType, Expression rowComparator,
+      Expression keySelector, Expression keyComparator) {
+    RexWindowBound bound = lower ? group.lowerBound : group.upperBound;
+    if (bound.isUnbounded()) {
+      return bound.isPreceding() ? min_ : max_;
+    }
+    if (group.isRows) {
+      if (bound.isCurrentRow()) {
+        return i_;
+      }
+      RexNode node = bound.getOffset();
+      Expression offs = translator.translate(node);
+      // Floating offset does not make sense since we refer to array index.
+      // Nulls do not make sense as well.
+      offs = RexToLixTranslator.convert(offs, int.class);
+
+      Expression b = i_;
+      if (bound.isFollowing()) {
+        b = Expressions.add(b, offs);
+      } else {
+        b = Expressions.subtract(b, offs);
+      }
+      return b;
+    }
+    Expression searchLower = min_;
+    Expression searchUpper = max_;
+    if (bound.isCurrentRow()) {
+      if (lower) {
+        searchUpper = i_;
+      } else {
+        searchLower = i_;
+      }
+    }
+
+    List<RelFieldCollation> fieldCollations =
+        group.collation().getFieldCollations();
+    if (bound.isCurrentRow() && fieldCollations.size() != 1) {
+      return Expressions.call(
+          (lower
+              ? BuiltInMethod.BINARY_SEARCH5_LOWER
+              : BuiltInMethod.BINARY_SEARCH5_UPPER).method,
+          rows_, row_, searchLower, searchUpper, keySelector, keyComparator);
+    }
+    assert fieldCollations.size() == 1
+        : "When using range window specification, ORDER BY should have"
+        + " exactly one expression."
+        + " Actual collation is " + group.collation();
+    // isRange
+    int orderKey =
+        fieldCollations.get(0).getFieldIndex();
+    RelDataType keyType =
+        physType.getRowType().getFieldList().get(orderKey).getType();
+    Type desiredKeyType = translator.typeFactory.getJavaClass(keyType);
+    if (bound.getOffset() == null) {
+      desiredKeyType = Primitive.box(desiredKeyType);
+    }
+    Expression val = translator.translate(new RexInputRef(orderKey,
+            keyType), desiredKeyType);
+    if (!bound.isCurrentRow()) {
+      RexNode node = bound.getOffset();
+      Expression offs = translator.translate(node);
+      // TODO: support date + interval somehow
+      if (bound.isFollowing()) {
+        val = Expressions.add(val, offs);
+      } else {
+        val = Expressions.subtract(val, offs);
+      }
+    }
+    return Expressions.call(
+        (lower
+            ? BuiltInMethod.BINARY_SEARCH6_LOWER
+            : BuiltInMethod.BINARY_SEARCH6_UPPER).method,
+        rows_, val, searchLower, searchUpper, keySelector, keyComparator);
+  }
+}
+
+// End EnumerableWindow.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableWindowRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableWindowRule.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableWindowRule.java
new file mode 100644
index 0000000..58e199f
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableWindowRule.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.logical.LogicalWindow;
+
+/**
+ * Rule to convert a {@link org.apache.calcite.rel.logical.LogicalAggregate} to
+ * an {@link org.apache.calcite.adapter.enumerable.EnumerableAggregate}.
+ */
+class EnumerableWindowRule extends ConverterRule {
+  EnumerableWindowRule() {
+    super(LogicalWindow.class, Convention.NONE, EnumerableConvention.INSTANCE,
+        "EnumerableWindowRule");
+  }
+
+  public RelNode convert(RelNode rel) {
+    final LogicalWindow winAgg = (LogicalWindow) rel;
+    final RelTraitSet traitSet =
+        winAgg.getTraitSet().replace(EnumerableConvention.INSTANCE);
+    final RelNode child = winAgg.getInput();
+    final RelNode convertedChild =
+        convert(child,
+            child.getTraitSet().replace(EnumerableConvention.INSTANCE));
+    return new EnumerableWindow(rel.getCluster(), traitSet, convertedChild,
+        winAgg.getConstants(), winAgg.getRowType(), winAgg.groups);
+  }
+}
+
+// End EnumerableWindowRule.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/JavaRelImplementor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/JavaRelImplementor.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/JavaRelImplementor.java
index 26e728e..b96efb8 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/JavaRelImplementor.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/JavaRelImplementor.java
@@ -14,15 +14,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.rules.java;
+package org.apache.calcite.adapter.enumerable;
 
-import net.hydromatic.linq4j.expressions.ParameterExpression;
-
-import net.hydromatic.optiq.DataContext;
-import net.hydromatic.optiq.impl.java.JavaTypeFactory;
-
-import org.eigenbase.rel.RelImplementorImpl;
-import org.eigenbase.rex.RexBuilder;
+import org.apache.calcite.DataContext;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.linq4j.tree.ParameterExpression;
+import org.apache.calcite.rel.RelImplementorImpl;
+import org.apache.calcite.rex.RexBuilder;
 
 /**
  * Abstract base class for implementations of {@link RelImplementorImpl}
@@ -33,13 +31,12 @@ public abstract class JavaRelImplementor extends RelImplementorImpl {
     super(rexBuilder);
   }
 
-  @Override
-  public JavaTypeFactory getTypeFactory() {
+  @Override public JavaTypeFactory getTypeFactory() {
     return (JavaTypeFactory) super.getTypeFactory();
   }
 
   /** Returns the expression with which to access the
-   * {@link net.hydromatic.optiq.DataContext}. */
+   * {@link org.apache.calcite.DataContext}. */
   public ParameterExpression getRootExpression() {
     return DataContext.ROOT;
   }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/JavaRowFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/JavaRowFormat.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/JavaRowFormat.java
index 6773ff3..d1a119c 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/JavaRowFormat.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/JavaRowFormat.java
@@ -14,16 +14,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.rules.java;
+package org.apache.calcite.adapter.enumerable;
 
-import net.hydromatic.linq4j.expressions.*;
-
-import net.hydromatic.optiq.BuiltinMethod;
-import net.hydromatic.optiq.impl.java.JavaTypeFactory;
-import net.hydromatic.optiq.runtime.FlatLists;
-import net.hydromatic.optiq.runtime.Unit;
-
-import org.eigenbase.reltype.RelDataType;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.ExpressionType;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.linq4j.tree.MemberExpression;
+import org.apache.calcite.linq4j.tree.Types;
+import org.apache.calcite.linq4j.tree.UnaryExpression;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.runtime.FlatLists;
+import org.apache.calcite.runtime.Unit;
+import org.apache.calcite.util.BuiltInMethod;
 
 import java.lang.reflect.Type;
 import java.util.AbstractList;
@@ -41,8 +44,7 @@ public enum JavaRowFormat {
       return typeFactory.getJavaClass(type);
     }
 
-    @Override
-    Type javaFieldClass(JavaTypeFactory typeFactory, RelDataType type,
+    @Override Type javaFieldClass(JavaTypeFactory typeFactory, RelDataType type,
         int index) {
       return typeFactory.getJavaClass(type.getFieldList().get(index).getType());
     }
@@ -58,8 +60,7 @@ public enum JavaRowFormat {
       }
     }
 
-    @Override
-    public MemberExpression field(Expression expression, int field,
+    @Override public MemberExpression field(Expression expression, int field,
         Type fieldType) {
       final Type type = expression.getType();
       if (type instanceof Types.RecordType) {
@@ -85,8 +86,7 @@ public enum JavaRowFormat {
           type.getFieldList().get(0).getType());
     }
 
-    @Override
-    Type javaFieldClass(JavaTypeFactory typeFactory, RelDataType type,
+    @Override Type javaFieldClass(JavaTypeFactory typeFactory, RelDataType type,
         int index) {
       return javaRowClass(typeFactory, type);
     }
@@ -97,8 +97,8 @@ public enum JavaRowFormat {
       return expressions.get(0);
     }
 
-    @Override
-    public Expression field(Expression expression, int field, Type fieldType) {
+    @Override public Expression field(Expression expression, int field,
+        Type fieldType) {
       assert field == 0;
       return expression;
     }
@@ -114,8 +114,7 @@ public enum JavaRowFormat {
       return FlatLists.ComparableList.class;
     }
 
-    @Override
-    Type javaFieldClass(JavaTypeFactory typeFactory, RelDataType type,
+    @Override Type javaFieldClass(JavaTypeFactory typeFactory, RelDataType type,
         int index) {
       return Object.class;
     }
@@ -133,7 +132,7 @@ public enum JavaRowFormat {
             Expressions.call(
                 List.class,
                 null,
-                BuiltinMethod.LIST2.method,
+                BuiltInMethod.LIST2.method,
                 expressions),
             List.class);
       case 3:
@@ -141,7 +140,7 @@ public enum JavaRowFormat {
             Expressions.call(
                 List.class,
                 null,
-                BuiltinMethod.LIST3.method,
+                BuiltInMethod.LIST3.method,
                 expressions),
             List.class);
       default:
@@ -149,7 +148,7 @@ public enum JavaRowFormat {
             Expressions.call(
                 List.class,
                 null,
-                BuiltinMethod.ARRAYS_AS_LIST.method,
+                BuiltInMethod.ARRAYS_AS_LIST.method,
                 Expressions.newArrayInit(
                     Object.class,
                     expressions)),
@@ -157,11 +156,11 @@ public enum JavaRowFormat {
       }
     }
 
-    @Override
-    public Expression field(Expression expression, int field, Type fieldType) {
+    @Override public Expression field(Expression expression, int field,
+        Type fieldType) {
       return RexToLixTranslator.convert(
           Expressions.call(expression,
-              BuiltinMethod.LIST_GET.method,
+              BuiltInMethod.LIST_GET.method,
               Expressions.constant(field)),
           fieldType);
     }
@@ -175,8 +174,7 @@ public enum JavaRowFormat {
       return Object[].class;
     }
 
-    @Override
-    Type javaFieldClass(JavaTypeFactory typeFactory, RelDataType type,
+    @Override Type javaFieldClass(JavaTypeFactory typeFactory, RelDataType type,
         int index) {
       return Object.class;
     }
@@ -188,11 +186,11 @@ public enum JavaRowFormat {
     }
 
     @Override public Expression comparer() {
-      return Expressions.call(BuiltinMethod.ARRAY_COMPARER.method);
+      return Expressions.call(BuiltInMethod.ARRAY_COMPARER.method);
     }
 
-    @Override
-    public Expression field(Expression expression, int field, Type fieldType) {
+    @Override public Expression field(Expression expression, int field,
+        Type fieldType) {
       return RexToLixTranslator.convert(
           Expressions.arrayIndex(expression, Expressions.constant(field)),
           fieldType);

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/NestedBlockBuilder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/NestedBlockBuilder.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/NestedBlockBuilder.java
index e97d621..c78a25c 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/NestedBlockBuilder.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/NestedBlockBuilder.java
@@ -14,18 +14,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.rules.java;
+package org.apache.calcite.adapter.enumerable;
 
-import net.hydromatic.linq4j.expressions.BlockBuilder;
-
-import org.eigenbase.rex.RexNode;
+import org.apache.calcite.linq4j.tree.BlockBuilder;
+import org.apache.calcite.rex.RexNode;
 
 import java.util.Map;
 
 /**
- * Allows to build nested code blocks with tracking of current context and
- * the nullability of particular {@link org.eigenbase.rex.RexNode} expressions.
- * @see net.hydromatic.optiq.rules.java.StrictAggImplementor#implementAdd(AggContext, AggAddContext)
+ * Allows to build nested code blocks with tracking of current context and the
+ * nullability of particular {@link org.apache.calcite.rex.RexNode} expressions.
+ *
+ * @see org.apache.calcite.adapter.enumerable.StrictAggImplementor#implementAdd(AggContext, AggAddContext)
  */
 public interface NestedBlockBuilder {
   /**

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/NestedBlockBuilderImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/NestedBlockBuilderImpl.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/NestedBlockBuilderImpl.java
index 5f50f17..1caa47e 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/NestedBlockBuilderImpl.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/NestedBlockBuilderImpl.java
@@ -14,18 +14,22 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.rules.java;
+package org.apache.calcite.adapter.enumerable;
 
-import net.hydromatic.linq4j.expressions.BlockBuilder;
+import org.apache.calcite.linq4j.tree.BlockBuilder;
+import org.apache.calcite.rex.RexNode;
 
-import org.eigenbase.rex.RexNode;
-
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
 
 /**
- * Allows to build nested code blocks with tracking of current context and
- * the nullability of particular {@link org.eigenbase.rex.RexNode} expressions.
- * @see net.hydromatic.optiq.rules.java.StrictAggImplementor#implementAdd(AggContext, AggAddContext)
+ * Allows to build nested code blocks with tracking of current context and the
+ * nullability of particular {@link org.apache.calcite.rex.RexNode} expressions.
+ *
+ * @see org.apache.calcite.adapter.enumerable.StrictAggImplementor#implementAdd(AggContext, AggAddContext)
  */
 public class NestedBlockBuilderImpl implements NestedBlockBuilder {
   private final List<BlockBuilder> blocks = new ArrayList<BlockBuilder>();
@@ -113,4 +117,4 @@ public class NestedBlockBuilderImpl implements NestedBlockBuilder {
   }
 }
 
-// End NestedBlockBuilder.java
+// End NestedBlockBuilderImpl.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/NotNullImplementor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/NotNullImplementor.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/NotNullImplementor.java
index 8926a4e..0144d3a 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/NotNullImplementor.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/NotNullImplementor.java
@@ -14,20 +14,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.rules.java;
+package org.apache.calcite.adapter.enumerable;
 
-import net.hydromatic.linq4j.expressions.Expression;
-
-import org.eigenbase.rex.RexCall;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.rex.RexCall;
 
 import java.util.List;
 
 /**
- * Simplified version of {@link net.hydromatic.optiq.rules.java.CallImplementor}
+ * Simplified version of
+ * {@link org.apache.calcite.adapter.enumerable.CallImplementor}
  * that does not know about null semantics.
  *
- * @see net.hydromatic.optiq.rules.java.RexImpTable
- * @see net.hydromatic.optiq.rules.java.CallImplementor
+ * @see org.apache.calcite.adapter.enumerable.RexImpTable
+ * @see org.apache.calcite.adapter.enumerable.CallImplementor
  */
 public interface NotNullImplementor {
   /**

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/NullPolicy.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/NullPolicy.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/NullPolicy.java
index 7e98302..dedf300 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/NullPolicy.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/NullPolicy.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.rules.java;
+package org.apache.calcite.adapter.enumerable;
 
 /**
  * Describes when a function/operator will return null.
@@ -40,3 +40,5 @@ public enum NullPolicy {
   NOT,
   NONE
 }
+
+// End NullPolicy.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/PhysType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/PhysType.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/PhysType.java
index 51e60a0..e620c13 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/PhysType.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/PhysType.java
@@ -14,15 +14,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.rules.java;
-
-import net.hydromatic.linq4j.expressions.Expression;
-import net.hydromatic.linq4j.expressions.ParameterExpression;
-
-import org.eigenbase.rel.RelCollation;
-import org.eigenbase.rel.RelFieldCollation;
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.util.Pair;
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.ParameterExpression;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.util.Pair;
 
 import java.lang.reflect.Type;
 import java.util.List;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/PhysTypeImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/PhysTypeImpl.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/PhysTypeImpl.java
index e30ce21..562e4ba 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/PhysTypeImpl.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/PhysTypeImpl.java
@@ -14,22 +14,26 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.rules.java;
-
-import net.hydromatic.linq4j.expressions.*;
-import net.hydromatic.linq4j.function.Function1;
-
-import net.hydromatic.optiq.BuiltinMethod;
-import net.hydromatic.optiq.impl.java.JavaTypeFactory;
-import net.hydromatic.optiq.runtime.Utilities;
-
-import org.eigenbase.rel.RelCollation;
-import org.eigenbase.rel.RelFieldCollation;
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.reltype.RelDataTypeFactory;
-import org.eigenbase.reltype.RelDataTypeField;
-import org.eigenbase.util.Pair;
-import org.eigenbase.util.Util;
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.linq4j.function.Function1;
+import org.apache.calcite.linq4j.tree.BlockBuilder;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.linq4j.tree.MemberDeclaration;
+import org.apache.calcite.linq4j.tree.ParameterExpression;
+import org.apache.calcite.linq4j.tree.Primitive;
+import org.apache.calcite.linq4j.tree.Types;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.runtime.Utilities;
+import org.apache.calcite.util.BuiltInMethod;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
 
 import com.google.common.base.Function;
 import com.google.common.collect.ImmutableList;
@@ -37,7 +41,14 @@ import com.google.common.collect.Lists;
 
 import java.lang.reflect.Modifier;
 import java.lang.reflect.Type;
-import java.util.*;
+import java.util.AbstractList;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+
+import static org.apache.calcite.adapter.enumerable.EnumUtils.javaRowClass;
+import static org.apache.calcite.adapter.enumerable.EnumUtils.overridingMethodDecl;
 
 /** Implementation of {@link PhysType}. */
 public class PhysTypeImpl implements PhysType {
@@ -58,8 +69,7 @@ public class PhysTypeImpl implements PhysType {
     this.javaRowClass = javaRowClass;
     this.format = format;
     for (RelDataTypeField field : rowType.getFieldList()) {
-      fieldClasses.add(
-          JavaRules.EnumUtil.javaRowClass(typeFactory, field.getType()));
+      fieldClasses.add(javaRowClass(typeFactory, field.getType()));
     }
   }
 
@@ -85,8 +95,8 @@ public class PhysTypeImpl implements PhysType {
     }
     RelDataType rowType = builder.build();
     // Do not optimize if there are 0 or 1 fields.
-    return new PhysTypeImpl(
-        typeFactory, rowType, javaRowClass, JavaRowFormat.CUSTOM);
+    return new PhysTypeImpl(typeFactory, rowType, javaRowClass,
+        JavaRowFormat.CUSTOM);
   }
 
   public JavaRowFormat getFormat() {
@@ -128,7 +138,7 @@ public class PhysTypeImpl implements PhysType {
         project(fields, targetFormat);
     switch (format) {
     case SCALAR:
-      return Expressions.call(BuiltinMethod.IDENTITY_SELECTOR.method);
+      return Expressions.call(BuiltInMethod.IDENTITY_SELECTOR.method);
     default:
       return Expressions.lambda(Function1.class,
           targetPhysType.record(fieldReferences(parameter, fields)), parameter);
@@ -186,7 +196,7 @@ public class PhysTypeImpl implements PhysType {
     final ParameterExpression o_ =
         Expressions.parameter(javaRowClass, "o");
     final int fieldCount = rowType.getFieldCount();
-    return Expressions.call(exp, BuiltinMethod.SELECT.method,
+    return Expressions.call(exp, BuiltInMethod.SELECT.method,
         generateSelector(o_, Util.range(fieldCount), targetFormat));
   }
 
@@ -205,7 +215,7 @@ public class PhysTypeImpl implements PhysType {
       return Pair.<Expression, Expression>of(
           selector,
           Expressions.call(
-              BuiltinMethod.NULLS_COMPARATOR.method,
+              BuiltInMethod.NULLS_COMPARATOR.method,
               Expressions.constant(
                   collation.nullDirection
                       == RelFieldCollation.NullDirection.FIRST),
@@ -214,7 +224,7 @@ public class PhysTypeImpl implements PhysType {
                       == RelFieldCollation.Direction.DESCENDING)));
     }
     selector =
-        Expressions.call(BuiltinMethod.IDENTITY_SELECTOR.method);
+        Expressions.call(BuiltInMethod.IDENTITY_SELECTOR.method);
 
     // int c;
     // c = Utilities.compare(v0, v1);
@@ -281,7 +291,7 @@ public class PhysTypeImpl implements PhysType {
                     parameterV0, parameterV1),
                 body.toBlock()));
 
-    if (JavaRules.BRIDGE_METHODS) {
+    if (EnumerableRules.BRIDGE_METHODS) {
       final ParameterExpression parameterO0 =
           Expressions.parameter(Object.class, "o0");
       final ParameterExpression parameterO1 =
@@ -293,7 +303,7 @@ public class PhysTypeImpl implements PhysType {
               Expressions.call(
                   Expressions.parameter(
                       Comparable.class, "this"),
-                  BuiltinMethod.COMPARATOR_COMPARE.method,
+                  BuiltInMethod.COMPARATOR_COMPARE.method,
                   Expressions.convert_(
                       parameterO0,
                       javaRowClass),
@@ -301,8 +311,8 @@ public class PhysTypeImpl implements PhysType {
                       parameterO1,
                       javaRowClass))));
       memberDeclarations.add(
-          JavaRules.EnumUtil.overridingMethodDecl(
-              BuiltinMethod.COMPARATOR_COMPARE.method,
+          overridingMethodDecl(
+              BuiltInMethod.COMPARATOR_COMPARE.method,
               ImmutableList.of(parameterO0, parameterO1),
               bridgeBody.toBlock()));
     }
@@ -381,7 +391,7 @@ public class PhysTypeImpl implements PhysType {
                 ImmutableList.of(parameterV0, parameterV1),
                 body.toBlock()));
 
-    if (JavaRules.BRIDGE_METHODS) {
+    if (EnumerableRules.BRIDGE_METHODS) {
       final ParameterExpression parameterO0 =
           Expressions.parameter(Object.class, "o0");
       final ParameterExpression parameterO1 =
@@ -393,7 +403,7 @@ public class PhysTypeImpl implements PhysType {
               Expressions.call(
                   Expressions.parameter(
                       Comparable.class, "this"),
-                  BuiltinMethod.COMPARATOR_COMPARE.method,
+                  BuiltInMethod.COMPARATOR_COMPARE.method,
                   Expressions.convert_(
                       parameterO0,
                       javaRowClass),
@@ -401,8 +411,8 @@ public class PhysTypeImpl implements PhysType {
                       parameterO1,
                       javaRowClass))));
       memberDeclarations.add(
-          JavaRules.EnumUtil.overridingMethodDecl(
-              BuiltinMethod.COMPARATOR_COMPARE.method,
+          overridingMethodDecl(
+              BuiltInMethod.COMPARATOR_COMPARE.method,
               ImmutableList.of(parameterO0, parameterO1),
               bridgeBody.toBlock()));
     }
@@ -501,7 +511,7 @@ public class PhysTypeImpl implements PhysType {
             Expressions.call(
                 List.class,
                 null,
-                BuiltinMethod.LIST2.method,
+                BuiltInMethod.LIST2.method,
                 list),
             v1);
       case 3:
@@ -510,7 +520,7 @@ public class PhysTypeImpl implements PhysType {
             Expressions.call(
                 List.class,
                 null,
-                BuiltinMethod.LIST3.method,
+                BuiltInMethod.LIST3.method,
                 list),
             v1);
       default:
@@ -519,7 +529,7 @@ public class PhysTypeImpl implements PhysType {
             Expressions.call(
                 List.class,
                 null,
-                BuiltinMethod.ARRAYS_AS_LIST.method,
+                BuiltInMethod.ARRAYS_AS_LIST.method,
                 Expressions.newArrayInit(
                     Object.class,
                     list)),

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/ReflectiveCallNotNullImplementor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/ReflectiveCallNotNullImplementor.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/ReflectiveCallNotNullImplementor.java
index 527facf..bf35032 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/ReflectiveCallNotNullImplementor.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/ReflectiveCallNotNullImplementor.java
@@ -14,13 +14,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.rules.java;
+package org.apache.calcite.adapter.enumerable;
 
-import net.hydromatic.linq4j.expressions.Expression;
-import net.hydromatic.linq4j.expressions.Expressions;
-import net.hydromatic.linq4j.expressions.NewExpression;
-
-import org.eigenbase.rex.RexCall;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.linq4j.tree.NewExpression;
+import org.apache.calcite.rex.RexCall;
 
 import java.lang.reflect.Method;
 import java.lang.reflect.Modifier;
@@ -28,7 +27,7 @@ import java.util.List;
 
 /**
  * Implementation of
- * {@link net.hydromatic.optiq.rules.java.NotNullImplementor}
+ * {@link org.apache.calcite.adapter.enumerable.NotNullImplementor}
  * that calls a given {@link java.lang.reflect.Method}.
  *
  * <p>When method is not static, a new instance of the required class is


[29/58] [abbrv] [partial] incubator-calcite git commit: [CALCITE-306] Standardize code style for "import package.*; "

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rex/RexTransformer.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexTransformer.java b/core/src/main/java/org/apache/calcite/rex/RexTransformer.java
index 167118e..307b466 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexTransformer.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexTransformer.java
@@ -14,14 +14,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rex;
+package org.apache.calcite.rex;
 
-import java.util.*;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.type.SqlTypeUtil;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.fun.*;
-import org.eigenbase.sql.type.*;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Set;
 
 /**
  * Takes a tree of {@link RexNode} objects and transforms it into another in one
@@ -51,7 +54,7 @@ public class RexTransformer {
     transformableOperators.add(SqlStdOperatorTable.AND);
 
     /** NOTE the OR operator is NOT missing.
-     * see {@link org.eigenbase.test.RexTransformerTest} */
+     * see {@link org.apache.calcite.test.RexTransformerTest} */
     transformableOperators.add(SqlStdOperatorTable.EQUALS);
     transformableOperators.add(SqlStdOperatorTable.NOT_EQUALS);
     transformableOperators.add(SqlStdOperatorTable.GREATER_THAN);

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 af365e1..286f8fb 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexUtil.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexUtil.java
@@ -14,20 +14,26 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rex;
-
-import java.util.*;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.fun.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.util.*;
-import org.eigenbase.util.mapping.*;
-
-import net.hydromatic.linq4j.function.*;
+package org.apache.calcite.rex;
+
+import org.apache.calcite.linq4j.function.Predicate1;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollationImpl;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.sql.SqlAggFunction;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlOperator;
+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.Pair;
+import org.apache.calcite.util.Util;
+import org.apache.calcite.util.mapping.Mappings;
 
 import com.google.common.base.Function;
 import com.google.common.collect.ImmutableList;
@@ -35,6 +41,11 @@ import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
 /**
  * Utility methods concerning row-expressions.
  */
@@ -140,7 +151,7 @@ public class RexUtil {
    * <p>Examples:
    *
    * <ul>
-   * <li>For {@link org.eigenbase.rex.RexLiteral} Unknown, returns false.
+   * <li>For {@link org.apache.calcite.rex.RexLiteral} Unknown, returns false.
    * <li>For <code>CAST(NULL AS <i>type</i>)</code>, returns true if <code>
    * allowCast</code> is true, false otherwise.
    * <li>For <code>CAST(CAST(NULL AS <i>type</i>) AS <i>type</i>))</code>,
@@ -240,8 +251,8 @@ public class RexUtil {
   }
 
   /**
-   * Returns whether a given tree contains any {@link
-   * org.eigenbase.rex.RexFieldAccess} nodes.
+   * Returns whether a given tree contains any
+   * {@link org.apache.calcite.rex.RexFieldAccess} nodes.
    *
    * @param node a RexNode tree
    */
@@ -998,8 +1009,7 @@ public class RexUtil {
   public static RexNode shift(RexNode node, final int offset) {
     return node.accept(
         new RexShuttle() {
-          @Override
-          public RexNode visitInputRef(RexInputRef input) {
+          @Override public RexNode visitInputRef(RexInputRef input) {
             return new RexInputRef(input.getIndex() + offset, input.getType());
           }
         });
@@ -1012,8 +1022,7 @@ public class RexUtil {
   public static RexNode shift(RexNode node, final int start, final int offset) {
     return node.accept(
         new RexShuttle() {
-          @Override
-          public RexNode visitInputRef(RexInputRef input) {
+          @Override public RexNode visitInputRef(RexInputRef input) {
             final int index = input.getIndex();
             if (index < start) {
               return input;
@@ -1141,10 +1150,10 @@ public class RexUtil {
   }
 
   /**
-   * Walks over an expression and throws an exception if it finds an {@link
-   * RexInputRef} with an ordinal beyond the number of fields in the input row
-   * type, or a {@link RexLocalRef} with ordinal greater than that set using
-   * {@link #setLimit(int)}.
+   * Walks over an expression and throws an exception if it finds an
+   * {@link RexInputRef} with an ordinal beyond the number of fields in the
+   * input row type, or a {@link RexLocalRef} with ordinal greater than that set
+   * using {@link #setLimit(int)}.
    */
   private static class ForwardRefFinder extends RexVisitorImpl<Void> {
     private int limit = -1;
@@ -1175,6 +1184,8 @@ public class RexUtil {
       this.limit = limit;
     }
 
+    /** Thrown to abort a visit when we find an illegal forward reference.
+     * It changes control flow but is not considered an error. */
     static class IllegalForwardRefException extends ControlFlowException {
     }
   }
@@ -1207,7 +1218,7 @@ public class RexUtil {
     }
   }
 
-  /** Helps {@link org.eigenbase.rex.RexUtil#toCnf}. */
+  /** Helps {@link org.apache.calcite.rex.RexUtil#toCnf}. */
   private static class CnfHelper {
     final RexBuilder rexBuilder;
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rex/RexVariable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexVariable.java b/core/src/main/java/org/apache/calcite/rex/RexVariable.java
index 79c2e69..717a569 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexVariable.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexVariable.java
@@ -14,9 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rex;
+package org.apache.calcite.rex;
 
-import org.eigenbase.reltype.*;
+import org.apache.calcite.rel.type.RelDataType;
 
 /**
  * A row-expression which references a field.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rex/RexVisitor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexVisitor.java b/core/src/main/java/org/apache/calcite/rex/RexVisitor.java
index f22cc6e..fd3d6b6 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexVisitor.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexVisitor.java
@@ -14,14 +14,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rex;
+package org.apache.calcite.rex;
 
 /**
  * Visitor pattern for traversing a tree of {@link RexNode} objects.
  *
- * @see org.eigenbase.util.Glossary#VISITOR_PATTERN
+ * @see org.apache.calcite.util.Glossary#VISITOR_PATTERN
  * @see RexShuttle
  * @see RexVisitorImpl
+ *
+ * @param <R> Return type
  */
 public interface RexVisitor<R> {
   //~ Methods ----------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rex/RexVisitorImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexVisitorImpl.java b/core/src/main/java/org/apache/calcite/rex/RexVisitorImpl.java
index 5c86a7c..fcc6ff7 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexVisitorImpl.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexVisitorImpl.java
@@ -14,13 +14,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rex;
+package org.apache.calcite.rex;
 
 import java.util.List;
 
 /**
  * Default implementation of {@link RexVisitor}, which visits each node but does
  * nothing while it's there.
+ *
+ * @param <R> Return type from each {@code visitXxx} method.
  */
 public class RexVisitorImpl<R> implements RexVisitor<R> {
   //~ Instance fields --------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rex/RexWindow.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexWindow.java b/core/src/main/java/org/apache/calcite/rex/RexWindow.java
index dfffbed..8fe3008 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexWindow.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexWindow.java
@@ -14,13 +14,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rex;
-
-import java.io.*;
-import java.util.List;
+package org.apache.calcite.rex;
 
 import com.google.common.collect.ImmutableList;
 
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.util.List;
+
 /**
  * Specification of the window of rows over which a {@link RexOver} windowed
  * aggregate is evaluated.
@@ -42,8 +43,8 @@ public class RexWindow {
   /**
    * Creates a window.
    *
-   * <p>If you need to create a window from outside this package, use {@link
-   * RexBuilder#makeOver}.
+   * <p>If you need to create a window from outside this package, use
+   * {@link RexBuilder#makeOver}.
    */
   RexWindow(
       List<RexNode> partitionKeys,

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rex/RexWindowBound.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexWindowBound.java b/core/src/main/java/org/apache/calcite/rex/RexWindowBound.java
index e866d2f..846f9dd 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexWindowBound.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexWindowBound.java
@@ -14,12 +14,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rex;
+package org.apache.calcite.rex;
 
-import org.eigenbase.sql.SqlKind;
-import org.eigenbase.sql.SqlLiteral;
-import org.eigenbase.sql.SqlNode;
-import org.eigenbase.sql.SqlWindow;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlWindow;
 
 /**
  * Abstracts "XX PRECEDING/FOLLOWING" and "CURRENT ROW" bounds for windowed
@@ -94,7 +94,7 @@ public abstract class RexWindowBound {
   public int getOrderKey() { return -1; }
 
   /**
-   * Transforms the bound via {@link org.eigenbase.rex.RexVisitor}.
+   * Transforms the bound via {@link org.apache.calcite.rex.RexVisitor}.
    * @param visitor visitor to accept
    * @param <R> return type of the visitor
    * @return transformed bound
@@ -113,33 +113,27 @@ public abstract class RexWindowBound {
       this.node = node;
     }
 
-    @Override
-    public boolean isUnbounded() {
+    @Override public boolean isUnbounded() {
       return true;
     }
 
-    @Override
-    public boolean isPreceding() {
+    @Override public boolean isPreceding() {
       return SqlWindow.isUnboundedPreceding(node);
     }
 
-    @Override
-    public boolean isFollowing() {
+    @Override public boolean isFollowing() {
       return SqlWindow.isUnboundedFollowing(node);
     }
 
-    @Override
-    public String toString() {
+    @Override public String toString() {
       return ((SqlLiteral) node).getValue().toString();
     }
 
-    @Override
-    public int getOrderKey() {
+    @Override public int getOrderKey() {
       return isPreceding() ? 0 : 2;
     }
 
-    @Override
-    public boolean equals(Object o) {
+    @Override public boolean equals(Object o) {
       if (this == o) {
         return true;
       }
@@ -156,8 +150,7 @@ public abstract class RexWindowBound {
       return true;
     }
 
-    @Override
-    public int hashCode() {
+    @Override public int hashCode() {
       return node.hashCode();
     }
   }
@@ -166,28 +159,23 @@ public abstract class RexWindowBound {
    * Implements CURRENT ROW bound.
    */
   private static class RexWindowBoundCurrentRow extends RexWindowBound {
-    @Override
-    public boolean isCurrentRow() {
+    @Override public boolean isCurrentRow() {
       return true;
     }
 
-    @Override
-    public String toString() {
+    @Override public String toString() {
       return "CURRENT ROW";
     }
 
-    @Override
-    public int getOrderKey() {
+    @Override public int getOrderKey() {
       return 1;
     }
 
-    @Override
-    public boolean equals(Object obj) {
+    @Override public boolean equals(Object obj) {
       return getClass() == obj.getClass();
     }
 
-    @Override
-    public int hashCode() {
+    @Override public int hashCode() {
       return 123;
     }
   }
@@ -206,8 +194,8 @@ public abstract class RexWindowBound {
       RexCall call = (RexCall) node;
       this.offset = call.getOperands().get(0);
       this.sqlKind = call.getKind();
-      assert this.offset != null : "RexWindowBoundBounded offset should not be"
-                                   + " null";
+      assert this.offset != null
+          : "RexWindowBoundBounded offset should not be null";
     }
 
     private RexWindowBoundBounded(SqlKind sqlKind, RexNode offset) {
@@ -215,23 +203,19 @@ public abstract class RexWindowBound {
       this.offset = offset;
     }
 
-    @Override
-    public boolean isPreceding() {
+    @Override public boolean isPreceding() {
       return sqlKind == SqlKind.PRECEDING;
     }
 
-    @Override
-    public boolean isFollowing() {
+    @Override public boolean isFollowing() {
       return sqlKind == SqlKind.FOLLOWING;
     }
 
-    @Override
-    public RexNode getOffset() {
+    @Override public RexNode getOffset() {
       return offset;
     }
 
-    @Override
-    public <R> RexWindowBound accept(RexVisitor<R> visitor) {
+    @Override public <R> RexWindowBound accept(RexVisitor<R> visitor) {
       R r = offset.accept(visitor);
       if (r instanceof RexNode && r != offset) {
         return new RexWindowBoundBounded(sqlKind, (RexNode) r);
@@ -239,13 +223,11 @@ public abstract class RexWindowBound {
       return this;
     }
 
-    @Override
-    public String toString() {
+    @Override public String toString() {
       return offset.toString() + " " + sqlKind.toString();
     }
 
-    @Override
-    public boolean equals(Object o) {
+    @Override public boolean equals(Object o) {
       if (this == o) {
         return true;
       }
@@ -265,8 +247,7 @@ public abstract class RexWindowBound {
       return true;
     }
 
-    @Override
-    public int hashCode() {
+    @Override public int hashCode() {
       int result = sqlKind.hashCode();
       result = 31 * result + offset.hashCode();
       return result;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rex/package-info.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/package-info.java b/core/src/main/java/org/apache/calcite/rex/package-info.java
index 742790a..b776422 100644
--- a/core/src/main/java/org/apache/calcite/rex/package-info.java
+++ b/core/src/main/java/org/apache/calcite/rex/package-info.java
@@ -20,69 +20,65 @@
  *
  * <h2>Life-cycle</h2>
  *
- * <p>A {@link org.eigenbase.sql2rel.SqlToRelConverter} converts a SQL parse tree
- *     consisting of {@link org.eigenbase.sql.SqlNode} objects into a relational
- *     expression ({@link org.eigenbase.rel.RelNode}). Several kinds of nodes in
- *     this tree have row expressions ({@link org.eigenbase.rex.RexNode}).</p>
+ * <p>A {@link org.apache.calcite.sql2rel.SqlToRelConverter} converts a SQL
+ * parse tree consisting of {@link org.apache.calcite.sql.SqlNode} objects into
+ * a relational expression ({@link org.apache.calcite.rel.RelNode}). Several
+ * kinds of nodes in this tree have row expressions
+ * ({@link org.apache.calcite.rex.RexNode}).</p>
  *
  * <p>After the relational expression has been optimized, a
- *     {@link net.hydromatic.optiq.rules.java.JavaRelImplementor} converts it
- *     into to a plan. If the plan is a Java
- *     parse tree, row-expressions are translated into equivalent Java
- *     expressions.</p>
+ * {@link org.apache.calcite.adapter.enumerable.JavaRelImplementor} converts it
+ * into to a plan. If the plan is a Java parse tree, row-expressions are
+ * translated into equivalent Java expressions.</p>
  *
  * <h2>Expressions</h2>
  *
  *
  * <p>Every row-expression has a type. (Compare with
- *     {@link org.eigenbase.sql.SqlNode}, which is created before validation, and
- *     therefore types may not be available.)</p>
+ * {@link org.apache.calcite.sql.SqlNode}, which is created before validation,
+ * and therefore types may not be available.)</p>
  *
- * <p>Every node in the parse tree is a {@link org.eigenbase.rex.RexNode}.
+ * <p>Every node in the parse tree is a {@link org.apache.calcite.rex.RexNode}.
  *     Sub-types are:</p>
  * <ul>
- *     <li>{@link org.eigenbase.rex.RexLiteral} represents a boolean, numeric,
- *         string, or
- *         date constant, or the value <code>NULL</code>.
+ *     <li>{@link org.apache.calcite.rex.RexLiteral} represents a boolean,
+ *         numeric, string, or date constant, or the value <code>NULL</code>.
  *     </li>
- *     <li>{@link org.eigenbase.rex.RexVariable} represents a leaf of the tree. It
- *         has sub-types:
+ *     <li>{@link org.apache.calcite.rex.RexVariable} represents a leaf of the
+ *         tree. It has sub-types:
  *         <ul>
- *             <li>{@link org.eigenbase.rex.RexCorrelVariable} is a correlating
- *                 variable for
- *                 nested-loop joins
+ *             <li>{@link org.apache.calcite.rex.RexCorrelVariable} is a
+ *                 correlating variable for nested-loop joins
  *             </li>
- *             <li>{@link org.eigenbase.rex.RexInputRef} refers to a field of an
- *                 input
- *                 relational expression
+ *             <li>{@link org.apache.calcite.rex.RexInputRef} refers to a field
+ *                 of an input relational expression
  *             </li>
- *             <li>{@link org.eigenbase.rex.RexCall} is a call to an operator or
- *                 function.
- *                 By means of special operators, we can use this construct to
- *                 represent
- *                 virtually every non-leaf node in the tree.
+ *             <li>{@link org.apache.calcite.rex.RexCall} is a call to an
+ *                 operator or function.  By means of special operators, we can
+ *                 use this construct to represent virtually every non-leaf node
+ *                 in the tree.
  *             </li>
- *             <li>{@link org.eigenbase.rex.RexRangeRef} refers to a collection of
- *                 contiguous fields from an input relational expression. It
- *                 usually exists only
- *                 during translation.
+ *             <li>{@link org.apache.calcite.rex.RexRangeRef} refers to a
+ *                 collection of contiguous fields from an input relational
+ *                 expression. It usually exists only during translation.
  *             </li>
  *         </ul>
  *     </li>
  * </ul>
  *
  * <p>Expressions are generally
- *     created using a {@link org.eigenbase.rex.RexBuilder} factory.</p>
+ *     created using a {@link org.apache.calcite.rex.RexBuilder} factory.</p>
  *
  * <h2>Related packages</h2>
  * <ul>
- *     <li>{@link org.eigenbase.sql} SQL object model</li>
- *     <li>{@link org.eigenbase.relopt} Core classes, including {@link
- *         org.eigenbase.reltype.RelDataType} and {@link
- *         org.eigenbase.reltype.RelDataTypeFactory}.
- *     </li>
+ *     <li>{@link org.apache.calcite.sql} SQL object model</li>
+ *
+ *     <li>{@link org.apache.calcite.plan} Core classes, including
+ * {@link org.apache.calcite.rel.type.RelDataType} and
+ * {@link org.apache.calcite.rel.type.RelDataTypeFactory}.</li>
+ *
  * </ul>
  */
-package org.eigenbase.rex;
+package org.apache.calcite.rex;
 
 // End package-info.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rules/package-info.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rules/package-info.java b/core/src/main/java/org/apache/calcite/rules/package-info.java
deleted file mode 100644
index be29176..0000000
--- a/core/src/main/java/org/apache/calcite/rules/package-info.java
+++ /dev/null
@@ -1,23 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to you under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-/**
- * Query optimizer rules.
- */
-package net.hydromatic.optiq.rules;
-
-// End package-info.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/runtime/AbstractCursor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/runtime/AbstractCursor.java b/core/src/main/java/org/apache/calcite/runtime/AbstractCursor.java
index 24d404b..547fec6 100644
--- a/core/src/main/java/org/apache/calcite/runtime/AbstractCursor.java
+++ b/core/src/main/java/org/apache/calcite/runtime/AbstractCursor.java
@@ -14,29 +14,42 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.runtime;
+package org.apache.calcite.runtime;
 
-import net.hydromatic.avatica.*;
-
-import net.hydromatic.linq4j.expressions.Primitive;
-
-import org.eigenbase.util.Util;
-import org.eigenbase.util14.DateTimeUtil;
+import org.apache.calcite.avatica.ArrayImpl;
+import org.apache.calcite.avatica.ByteString;
+import org.apache.calcite.avatica.ColumnMetaData;
+import org.apache.calcite.avatica.Cursor;
+import org.apache.calcite.linq4j.tree.Primitive;
+import org.apache.calcite.util.DateTimeUtil;
+import org.apache.calcite.util.Util;
 
 import java.io.InputStream;
 import java.io.Reader;
 import java.math.BigDecimal;
 import java.math.RoundingMode;
 import java.net.URL;
-import java.sql.*;
+import java.sql.Array;
+import java.sql.Blob;
+import java.sql.Clob;
 import java.sql.Date;
-import java.util.*;
+import java.sql.NClob;
+import java.sql.Ref;
+import java.sql.SQLXML;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.sql.Types;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
 
 /**
  * Base class for implementing a cursor.
  *
  * <p>Derived class needs to provide {@link Getter} and can override
- * {@link net.hydromatic.avatica.Cursor.Accessor} implementations if it
+ * {@link org.apache.calcite.avatica.Cursor.Accessor} implementations if it
  * wishes.</p>
  */
 public abstract class AbstractCursor implements Cursor {
@@ -663,7 +676,7 @@ public abstract class AbstractCursor implements Cursor {
 
   /**
    * Accessor that assumes that the underlying value is an array of
-   * {@link net.hydromatic.avatica.ByteString} values;
+   * {@link org.apache.calcite.avatica.ByteString} values;
    * corresponds to {@link java.sql.Types#BINARY}
    * and {@link java.sql.Types#VARBINARY}.
    */
@@ -691,13 +704,11 @@ public abstract class AbstractCursor implements Cursor {
       this.localCalendar = localCalendar;
     }
 
-    @Override
-    public Object getObject() {
+    @Override public Object getObject() {
       return getDate(localCalendar);
     }
 
-    @Override
-    public Date getDate(Calendar calendar) {
+    @Override public Date getDate(Calendar calendar) {
       final int v = getInt();
       if (v == 0 && getter.wasNull()) {
         return null;
@@ -705,8 +716,7 @@ public abstract class AbstractCursor implements Cursor {
       return longToDate((long) v * DateTimeUtil.MILLIS_PER_DAY, calendar);
     }
 
-    @Override
-    public Timestamp getTimestamp(Calendar calendar) {
+    @Override public Timestamp getTimestamp(Calendar calendar) {
       final int v = getInt();
       if (v == 0 && getter.wasNull()) {
         return null;
@@ -714,8 +724,7 @@ public abstract class AbstractCursor implements Cursor {
       return longToTimestamp((long) v * DateTimeUtil.MILLIS_PER_DAY, calendar);
     }
 
-    @Override
-    public String getString() {
+    @Override public String getString() {
       final int v = getInt();
       if (v == 0 && wasNull()) {
         return null;
@@ -737,13 +746,11 @@ public abstract class AbstractCursor implements Cursor {
       this.localCalendar = localCalendar;
     }
 
-    @Override
-    public Object getObject() {
+    @Override public Object getObject() {
       return getTime(localCalendar);
     }
 
-    @Override
-    public Time getTime(Calendar calendar) {
+    @Override public Time getTime(Calendar calendar) {
       final int v = getInt();
       if (v == 0 && wasNull()) {
         return null;
@@ -751,8 +758,7 @@ public abstract class AbstractCursor implements Cursor {
       return intToTime(v, calendar);
     }
 
-    @Override
-    public Timestamp getTimestamp(Calendar calendar) {
+    @Override public Timestamp getTimestamp(Calendar calendar) {
       final long v = getLong();
       if (v == 0 && wasNull()) {
         return null;
@@ -760,8 +766,7 @@ public abstract class AbstractCursor implements Cursor {
       return longToTimestamp(v, calendar);
     }
 
-    @Override
-    public String getString() {
+    @Override public String getString() {
       final int v = getInt();
       if (v == 0 && wasNull()) {
         return null;
@@ -783,13 +788,11 @@ public abstract class AbstractCursor implements Cursor {
       this.localCalendar = localCalendar;
     }
 
-    @Override
-    public Object getObject() {
+    @Override public Object getObject() {
       return getTimestamp(localCalendar);
     }
 
-    @Override
-    public Timestamp getTimestamp(Calendar calendar) {
+    @Override public Timestamp getTimestamp(Calendar calendar) {
       final long v = getLong();
       if (v == 0 && wasNull()) {
         return null;
@@ -797,8 +800,7 @@ public abstract class AbstractCursor implements Cursor {
       return longToTimestamp(v, calendar);
     }
 
-    @Override
-    public String getString() {
+    @Override public String getString() {
       final long v = getLong();
       if (v == 0L && wasNull()) {
         return null;
@@ -820,8 +822,7 @@ public abstract class AbstractCursor implements Cursor {
       this.localCalendar = localCalendar;
     }
 
-    @Override
-    public Date getDate(Calendar calendar) {
+    @Override public Date getDate(Calendar calendar) {
       java.sql.Date date = (Date) getObject();
       if (date == null) {
         return null;
@@ -834,8 +835,7 @@ public abstract class AbstractCursor implements Cursor {
       return date;
     }
 
-    @Override
-    public String getString() {
+    @Override public String getString() {
       final int v = getInt();
       if (v == 0 && wasNull()) {
         return null;
@@ -843,8 +843,7 @@ public abstract class AbstractCursor implements Cursor {
       return dateAsString(v, null);
     }
 
-    @Override
-    public long getLong() {
+    @Override public long getLong() {
       Date date = getDate(null);
       return date == null
           ? 0L
@@ -865,8 +864,7 @@ public abstract class AbstractCursor implements Cursor {
       this.localCalendar = localCalendar;
     }
 
-    @Override
-    public Time getTime(Calendar calendar) {
+    @Override public Time getTime(Calendar calendar) {
       Time date  = (Time) getObject();
       if (date == null) {
         return null;
@@ -879,8 +877,7 @@ public abstract class AbstractCursor implements Cursor {
       return date;
     }
 
-    @Override
-    public String getString() {
+    @Override public String getString() {
       final int v = getInt();
       if (v == 0 && wasNull()) {
         return null;
@@ -888,8 +885,7 @@ public abstract class AbstractCursor implements Cursor {
       return timeAsString(v, null);
     }
 
-    @Override
-    public long getLong() {
+    @Override public long getLong() {
       Time time = getTime(null);
       return time == null ? 0L : (time.getTime() % DateTimeUtil.MILLIS_PER_DAY);
     }
@@ -908,8 +904,7 @@ public abstract class AbstractCursor implements Cursor {
       this.localCalendar = localCalendar;
     }
 
-    @Override
-    public Timestamp getTimestamp(Calendar calendar) {
+    @Override public Timestamp getTimestamp(Calendar calendar) {
       Timestamp date  = (Timestamp) getObject();
       if (date == null) {
         return null;
@@ -922,8 +917,7 @@ public abstract class AbstractCursor implements Cursor {
       return date;
     }
 
-    @Override
-    public String getString() {
+    @Override public String getString() {
       final long v = getLong();
       if (v == 0 && wasNull()) {
         return null;
@@ -931,8 +925,7 @@ public abstract class AbstractCursor implements Cursor {
       return timestampAsString(v, null);
     }
 
-    @Override
-    public long getLong() {
+    @Override public long getLong() {
       Timestamp timestamp = getTimestamp(null);
       return timestamp == null ? 0 : timestamp.getTime();
     }
@@ -952,8 +945,7 @@ public abstract class AbstractCursor implements Cursor {
       this.localCalendar = localCalendar;
     }
 
-    @Override
-    public Timestamp getTimestamp(Calendar calendar) {
+    @Override public Timestamp getTimestamp(Calendar calendar) {
       java.util.Date date  = (java.util.Date) getObject();
       if (date == null) {
         return null;
@@ -965,8 +957,7 @@ public abstract class AbstractCursor implements Cursor {
       return new Timestamp(v);
     }
 
-    @Override
-    public String getString() {
+    @Override public String getString() {
       java.util.Date date  = (java.util.Date) getObject();
       if (date == null) {
         return null;
@@ -974,8 +965,7 @@ public abstract class AbstractCursor implements Cursor {
       return timestampAsString(date.getTime(), null);
     }
 
-    @Override
-    public long getLong() {
+    @Override public long getLong() {
       Timestamp timestamp = getTimestamp(localCalendar);
       return timestamp == null ? 0 : timestamp.getTime();
     }
@@ -994,8 +984,7 @@ public abstract class AbstractCursor implements Cursor {
       this.range = range;
     }
 
-    @Override
-    public String getString() {
+    @Override public String getString() {
       final int v = getInt();
       if (v == 0 && wasNull()) {
         return null;
@@ -1019,8 +1008,7 @@ public abstract class AbstractCursor implements Cursor {
       this.scale = scale;
     }
 
-    @Override
-    public String getString() {
+    @Override public String getString() {
       final long v = getLong();
       if (v == 0 && wasNull()) {
         return null;
@@ -1044,8 +1032,7 @@ public abstract class AbstractCursor implements Cursor {
       this.factory = factory;
     }
 
-    @Override
-    public Object getObject() {
+    @Override public Object getObject() {
       final Object object = super.getObject();
       if (object == null || object instanceof List) {
         return object;
@@ -1055,8 +1042,7 @@ public abstract class AbstractCursor implements Cursor {
       return Primitive.asList(object);
     }
 
-    @Override
-    public Array getArray() {
+    @Override public Array getArray() {
       final List list = (List) getObject();
       if (list == null) {
         return null;
@@ -1064,8 +1050,7 @@ public abstract class AbstractCursor implements Cursor {
       return new ArrayImpl(list, componentType, factory);
     }
 
-    @Override
-    public String getString() {
+    @Override public String getString() {
       final List o = (List) getObject();
       if (o == null) {
         return null;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/runtime/ArrayComparator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/runtime/ArrayComparator.java b/core/src/main/java/org/apache/calcite/runtime/ArrayComparator.java
index 59a4cda..d2f57ef 100644
--- a/core/src/main/java/org/apache/calcite/runtime/ArrayComparator.java
+++ b/core/src/main/java/org/apache/calcite/runtime/ArrayComparator.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.runtime;
+package org.apache.calcite.runtime;
 
 import com.google.common.collect.Ordering;
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/runtime/ArrayEnumeratorCursor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/runtime/ArrayEnumeratorCursor.java b/core/src/main/java/org/apache/calcite/runtime/ArrayEnumeratorCursor.java
index a4c4bc3..5d09ae7 100644
--- a/core/src/main/java/org/apache/calcite/runtime/ArrayEnumeratorCursor.java
+++ b/core/src/main/java/org/apache/calcite/runtime/ArrayEnumeratorCursor.java
@@ -14,13 +14,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.runtime;
+package org.apache.calcite.runtime;
 
-import net.hydromatic.linq4j.Enumerator;
+import org.apache.calcite.linq4j.Enumerator;
 
 /**
- * Implementation of {@link net.hydromatic.avatica.Cursor} on top of an
- * {@link net.hydromatic.linq4j.Enumerator} that
+ * Implementation of {@link org.apache.calcite.avatica.Cursor} on top of an
+ * {@link org.apache.calcite.linq4j.Enumerator} that
  * returns an array of {@link Object} for each row.
  */
 public class ArrayEnumeratorCursor extends EnumeratorCursor<Object[]> {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/runtime/BinarySearch.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/runtime/BinarySearch.java b/core/src/main/java/org/apache/calcite/runtime/BinarySearch.java
index 18ed28d..5311434 100644
--- a/core/src/main/java/org/apache/calcite/runtime/BinarySearch.java
+++ b/core/src/main/java/org/apache/calcite/runtime/BinarySearch.java
@@ -14,10 +14,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.runtime;
+package org.apache.calcite.runtime;
 
-import net.hydromatic.linq4j.function.Function1;
-import net.hydromatic.linq4j.function.Functions;
+import org.apache.calcite.linq4j.function.Function1;
+import org.apache.calcite.linq4j.function.Functions;
 
 import java.util.Comparator;
 
@@ -230,7 +230,7 @@ public class BinarySearch {
       // code must guarantee the interval is reduced at each iteration
       assert imid > imin
           : "search interval should be reduced min=" + imin
-              + ", mid=" + imid + ", max=" + imax;
+          + ", mid=" + imid + ", max=" + imax;
       // note: 0 <= imin < imax implies imid will always be less than imax
 
       // reduce the search
@@ -271,3 +271,5 @@ public class BinarySearch {
     }
   }
 }
+
+// End BinarySearch.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/runtime/Bindable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/runtime/Bindable.java b/core/src/main/java/org/apache/calcite/runtime/Bindable.java
index 741e369..a5c62e9 100644
--- a/core/src/main/java/org/apache/calcite/runtime/Bindable.java
+++ b/core/src/main/java/org/apache/calcite/runtime/Bindable.java
@@ -14,11 +14,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.runtime;
+package org.apache.calcite.runtime;
 
-import net.hydromatic.linq4j.Enumerable;
-
-import net.hydromatic.optiq.DataContext;
+import org.apache.calcite.DataContext;
+import org.apache.calcite.linq4j.Enumerable;
 
 /**
  * Statement that can be bound to a {@link DataContext} and then executed.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/runtime/CalciteContextException.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/runtime/CalciteContextException.java b/core/src/main/java/org/apache/calcite/runtime/CalciteContextException.java
index c51e019..1dddc7b 100644
--- a/core/src/main/java/org/apache/calcite/runtime/CalciteContextException.java
+++ b/core/src/main/java/org/apache/calcite/runtime/CalciteContextException.java
@@ -14,17 +14,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.util;
+package org.apache.calcite.runtime;
 
 // NOTE:  This class gets compiled independently of everything else so that
 // resource generation can use reflection.  That means it must have no
-// dependencies on other Eigenbase code.
+// dependencies on other Calcite code.
 
 /**
  * Exception which contains information about the textual context of the causing
  * exception.
  */
-public class EigenbaseContextException extends EigenbaseException {
+public class CalciteContextException extends CalciteException {
   //~ Static fields/initializers ---------------------------------------------
 
   /**
@@ -49,18 +49,18 @@ public class EigenbaseContextException extends EigenbaseException {
   //~ Constructors -----------------------------------------------------------
 
   /**
-   * Creates a new EigenbaseContextException object. This constructor is for
+   * Creates a new CalciteContextException object. This constructor is for
    * use by the generated factory.
    *
    * @param message error message
    * @param cause   underlying cause, must not be null
    */
-  public EigenbaseContextException(String message, Throwable cause) {
+  public CalciteContextException(String message, Throwable cause) {
     this(message, cause, 0, 0, 0, 0);
   }
 
   /**
-   * Creates a new EigenbaseContextException object.
+   * Creates a new CalciteContextException object.
    *
    * @param message      error message
    * @param cause        underlying cause, must not be null
@@ -69,7 +69,7 @@ public class EigenbaseContextException extends EigenbaseException {
    * @param endPosLine   1-based end line number
    * @param endPosColumn 1-based end column number
    */
-  public EigenbaseContextException(
+  public CalciteContextException(
       String message,
       Throwable cause,
       int posLine,
@@ -82,14 +82,14 @@ public class EigenbaseContextException extends EigenbaseException {
   }
 
   /**
-   * Creates a new EigenbaseContextException object. This constructor is for
+   * Creates a new CalciteContextException object. This constructor is for
    * use by the generated factory.
    *
    * @param message   error message
    * @param cause     underlying cause, must not be null
    * @param inputText is the orginal SQL statement, may be null
    */
-  public EigenbaseContextException(
+  public CalciteContextException(
       String message,
       Throwable cause,
       String inputText) {
@@ -174,12 +174,11 @@ public class EigenbaseContextException extends EigenbaseException {
     this.originalStatement = originalStatement;
   }
 
-  @Override
-  public String getMessage() {
+  @Override public String getMessage() {
     // The superclass' message is the textual context information
     // for this exception, so we add in the underlying cause to the message
     return super.getMessage() + ": " + getCause().getMessage();
   }
 }
 
-// End EigenbaseContextException.java
+// End CalciteContextException.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/runtime/CalciteException.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/runtime/CalciteException.java b/core/src/main/java/org/apache/calcite/runtime/CalciteException.java
index 2e69df1..387e8da 100644
--- a/core/src/main/java/org/apache/calcite/runtime/CalciteException.java
+++ b/core/src/main/java/org/apache/calcite/runtime/CalciteException.java
@@ -14,21 +14,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.util;
+package org.apache.calcite.runtime;
 
-import java.util.logging.*;
+import java.util.logging.Logger;
 
 // NOTE:  This class gets compiled independently of everything else so that
 // resource generation can use reflection.  That means it must have no
-// dependencies on other Eigenbase code.
+// dependencies on other Calcite code.
 
 
 /**
  * Base class for all exceptions originating from Farrago.
  *
- * @see EigenbaseContextException
+ * @see CalciteContextException
  */
-public class EigenbaseException extends RuntimeException {
+public class CalciteException extends RuntimeException {
   //~ Static fields/initializers ---------------------------------------------
 
   /**
@@ -39,26 +39,26 @@ public class EigenbaseException extends RuntimeException {
   private static final long serialVersionUID = -1314522633397794178L;
 
   private static final Logger LOGGER =
-      Logger.getLogger(EigenbaseException.class.getName());
+      Logger.getLogger(CalciteException.class.getName());
 
   //~ Constructors -----------------------------------------------------------
 
   /**
-   * Creates a new EigenbaseException object.
+   * Creates a new CalciteException object.
    *
    * @param message error message
    * @param cause   underlying cause
    */
-  public EigenbaseException(
+  public CalciteException(
       String message,
       Throwable cause) {
     super(message, cause);
 
     // TODO: Force the caller to pass in a Logger as a trace argument for
     // better context.  Need to extend ResGen for this.
-    LOGGER.throwing("EigenbaseException", "constructor", this);
+    LOGGER.throwing("CalciteException", "constructor", this);
     LOGGER.severe(toString());
   }
 }
 
-// End EigenbaseException.java
+// End CalciteException.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java b/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java
index 82b06a1..8b49f65 100644
--- a/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java
+++ b/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java
@@ -14,26 +14,28 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.resource;
+package org.apache.calcite.runtime;
 
-import org.eigenbase.sql.validate.SqlValidatorException;
-import org.eigenbase.util.EigenbaseContextException;
-import org.eigenbase.util.EigenbaseException;
+import org.apache.calcite.sql.validate.SqlValidatorException;
 
-import static org.eigenbase.resource.Resources.*;
+import static org.apache.calcite.runtime.Resources.BaseMessage;
+import static org.apache.calcite.runtime.Resources.ExInst;
+import static org.apache.calcite.runtime.Resources.ExInstWithCause;
+import static org.apache.calcite.runtime.Resources.Inst;
+import static org.apache.calcite.runtime.Resources.Property;
 
 /**
- * Compiler-checked resources for the Eigenbase project.
+ * Compiler-checked resources for the Calcite project.
  */
-public interface EigenbaseNewResource {
+public interface CalciteResource {
   @BaseMessage("line {0,number,#}, column {1,number,#}")
   Inst parserContext(int a0, int a1);
 
   @BaseMessage("Illegal {0} literal {1}: {2}")
-  ExInst<EigenbaseException> illegalLiteral(String a0, String a1, String a2);
+  ExInst<CalciteException> illegalLiteral(String a0, String a1, String a2);
 
   @BaseMessage("Length of identifier ''{0}'' must be less than or equal to {1,number,#} characters")
-  ExInst<EigenbaseException> identifierTooLong(String a0, int a1);
+  ExInst<CalciteException> identifierTooLong(String a0, int a1);
 
   @BaseMessage("not in format ''{0}''")
   Inst badFormat(String a0);
@@ -43,53 +45,53 @@ public interface EigenbaseNewResource {
 
   @BaseMessage("Illegal INTERVAL literal {0}; at {1}")
   @Property(name = "SQLSTATE", value = "42000")
-  ExInst<EigenbaseException> illegalIntervalLiteral(String a0, String a1);
+  ExInst<CalciteException> illegalIntervalLiteral(String a0, String a1);
 
   @BaseMessage("Illegal expression. Was expecting \"(DATETIME - DATETIME) INTERVALQUALIFIER\"")
-  ExInst<EigenbaseException> illegalMinusDate();
+  ExInst<CalciteException> illegalMinusDate();
 
   @BaseMessage("Illegal overlaps expression. Was expecting expression on the form \"(DATETIME, EXPRESSION) OVERLAPS (DATETIME, EXPRESSION)\"")
-  ExInst<EigenbaseException> illegalOverlaps();
+  ExInst<CalciteException> illegalOverlaps();
 
   @BaseMessage("Non-query expression encountered in illegal context")
-  ExInst<EigenbaseException> illegalNonQueryExpression();
+  ExInst<CalciteException> illegalNonQueryExpression();
 
   @BaseMessage("Query expression encountered in illegal context")
-  ExInst<EigenbaseException> illegalQueryExpression();
+  ExInst<CalciteException> illegalQueryExpression();
 
   @BaseMessage("CURSOR expression encountered in illegal context")
-  ExInst<EigenbaseException> illegalCursorExpression();
+  ExInst<CalciteException> illegalCursorExpression();
 
   @BaseMessage("ORDER BY unexpected")
-  ExInst<EigenbaseException> illegalOrderBy();
+  ExInst<CalciteException> illegalOrderBy();
 
   @BaseMessage("Illegal binary string {0}")
-  ExInst<EigenbaseException> illegalBinaryString(String a0);
+  ExInst<CalciteException> illegalBinaryString(String a0);
 
   @BaseMessage("''FROM'' without operands preceding it is illegal")
-  ExInst<EigenbaseException> illegalFromEmpty();
+  ExInst<CalciteException> illegalFromEmpty();
 
   @BaseMessage("ROW expression encountered in illegal context")
-  ExInst<EigenbaseException> illegalRowExpression();
+  ExInst<CalciteException> illegalRowExpression();
 
   @BaseMessage("TABLESAMPLE percentage must be between 0 and 100, inclusive")
   @Property(name = "SQLSTATE", value = "2202H")
-  ExInst<EigenbaseException> invalidSampleSize();
+  ExInst<CalciteException> invalidSampleSize();
 
   @BaseMessage("Unknown character set ''{0}''")
-  ExInst<EigenbaseException> unknownCharacterSet(String a0);
+  ExInst<CalciteException> unknownCharacterSet(String a0);
 
   @BaseMessage("Failed to encode ''{0}'' in character set ''{1}''")
-  ExInst<EigenbaseException> charsetEncoding(String a0, String a1);
+  ExInst<CalciteException> charsetEncoding(String a0, String a1);
 
   @BaseMessage("UESCAPE ''{0}'' must be exactly one character")
-  ExInst<EigenbaseException> unicodeEscapeCharLength(String a0);
+  ExInst<CalciteException> unicodeEscapeCharLength(String a0);
 
   @BaseMessage("UESCAPE ''{0}'' may not be hex digit, whitespace, plus sign, or double quote")
-  ExInst<EigenbaseException> unicodeEscapeCharIllegal(String a0);
+  ExInst<CalciteException> unicodeEscapeCharIllegal(String a0);
 
   @BaseMessage("UESCAPE cannot be specified without Unicode literal introducer")
-  ExInst<EigenbaseException> unicodeEscapeUnexpected();
+  ExInst<CalciteException> unicodeEscapeUnexpected();
 
   @BaseMessage("Unicode escape sequence starting at character {0,number,#} is not exactly four hex digits")
   ExInst<SqlValidatorException> unicodeEscapeMalformed(int a0);
@@ -101,11 +103,11 @@ public interface EigenbaseNewResource {
   ExInst<SqlValidatorException> invalidArgCount(String a0, int a1);
 
   @BaseMessage("At line {0,number,#}, column {1,number,#}")
-  ExInstWithCause<EigenbaseContextException> validatorContextPoint(int a0,
+  ExInstWithCause<CalciteContextException> validatorContextPoint(int a0,
       int a1);
 
   @BaseMessage("From line {0,number,#}, column {1,number,#} to line {2,number,#}, column {3,number,#}")
-  ExInstWithCause<EigenbaseContextException> validatorContext(int a0, int a1,
+  ExInstWithCause<CalciteContextException> validatorContext(int a0, int a1,
       int a2,
       int a3);
 
@@ -186,11 +188,11 @@ public interface EigenbaseNewResource {
   ExInst<SqlValidatorException> illegalMixingOfTypes();
 
   @BaseMessage("Invalid compare. Comparing (collation, coercibility): ({0}, {1} with ({2}, {3}) is illegal")
-  ExInst<EigenbaseException> invalidCompare(String a0, String a1, String a2,
+  ExInst<CalciteException> invalidCompare(String a0, String a1, String a2,
       String a3);
 
   @BaseMessage("Invalid syntax. Two explicit different collations ({0}, {1}) are illegal")
-  ExInst<EigenbaseException> differentCollations(String a0, String a1);
+  ExInst<CalciteException> differentCollations(String a0, String a1);
 
   @BaseMessage("{0} is not comparable to {1}")
   ExInst<SqlValidatorException> typeNotComparable(String a0, String a1);
@@ -402,7 +404,7 @@ public interface EigenbaseNewResource {
   ExInst<SqlValidatorException> duplicateNameInColumnList(String a0);
 
   @BaseMessage("Internal error: {0}")
-  ExInst<EigenbaseException> internal(String a0);
+  ExInst<CalciteException> internal(String a0);
 
   @BaseMessage("Argument to function ''{0}'' must be a literal")
   ExInst<SqlValidatorException> argumentMustBeLiteral(String a0);
@@ -411,10 +413,10 @@ public interface EigenbaseNewResource {
   ExInst<SqlValidatorException> argumentMustBePositiveInteger(String a0);
 
   @BaseMessage("Validation Error: {0}")
-  ExInst<EigenbaseException> validationError(String a0);
+  ExInst<CalciteException> validationError(String a0);
 
   @BaseMessage("Locale ''{0}'' in an illegal format")
-  ExInst<EigenbaseException> illegalLocaleFormat(String a0);
+  ExInst<CalciteException> illegalLocaleFormat(String a0);
 
   @BaseMessage("Argument to function ''{0}'' must not be NULL")
   ExInst<SqlValidatorException> argumentMustNotBeNull(String a0);
@@ -426,27 +428,27 @@ public interface EigenbaseNewResource {
   ExInst<SqlValidatorException> dynamicParamIllegal();
 
   @BaseMessage("''{0}'' is not a valid boolean value")
-  ExInst<EigenbaseException> invalidBoolean(String a0);
+  ExInst<CalciteException> invalidBoolean(String a0);
 
   @BaseMessage("Argument to function ''{0}'' must be a valid precision between ''{1,number,#}'' and ''{2,number,#}''")
   ExInst<SqlValidatorException> argumentMustBeValidPrecision(String a0, int a1,
       int a2);
 
   @BaseMessage("Wrong arguments for table function ''{0}'' call. Expected ''{1}'', actual ''{2}''")
-  ExInst<EigenbaseException> illegalArgumentForTableFunctionCall(String a0,
+  ExInst<CalciteException> illegalArgumentForTableFunctionCall(String a0,
       String a1, String a2);
 
   @BaseMessage("''{0}'' is not a valid datetime format")
-  ExInst<EigenbaseException> invalidDatetimeFormat(String a0);
+  ExInst<CalciteException> invalidDatetimeFormat(String a0);
 
   @BaseMessage("Cannot explicitly insert value into IDENTITY column ''{0}'' which is ALWAYS GENERATED")
-  ExInst<EigenbaseException> insertIntoAlwaysGenerated(String a0);
+  ExInst<CalciteException> insertIntoAlwaysGenerated(String a0);
 
   @BaseMessage("Argument to function ''{0}'' must have a scale of 0")
-  ExInst<EigenbaseException> argumentMustHaveScaleZero(String a0);
+  ExInst<CalciteException> argumentMustHaveScaleZero(String a0);
 
   @BaseMessage("Statement preparation aborted")
-  ExInst<EigenbaseException> preparationAborted();
+  ExInst<CalciteException> preparationAborted();
 
   @BaseMessage("SELECT DISTINCT not supported")
   @Property(name = "FeatureDefinition", value = "SQL:2003 Part 2 Annex F")
@@ -482,7 +484,7 @@ public interface EigenbaseNewResource {
 
   @BaseMessage("Execution of a new autocommit statement while a cursor is still open on same connection is not supported")
   @Property(name = "FeatureDefinition", value = "Eigenbase-defined")
-  ExInst<EigenbaseException>
+  ExInst<CalciteException>
   sQLConformance_MultipleActiveAutocommitStatements();
 
   @BaseMessage("Descending sort (ORDER BY DESC) not supported")
@@ -491,7 +493,7 @@ public interface EigenbaseNewResource {
 
   @BaseMessage("Sharing of cached statement plans not supported")
   @Property(name = "FeatureDefinition", value = "Eigenbase-defined")
-  ExInst<EigenbaseException> sharedStatementPlans();
+  ExInst<CalciteException> sharedStatementPlans();
 
   @BaseMessage("TABLESAMPLE SUBSTITUTE not supported")
   @Property(name = "FeatureDefinition", value = "Eigenbase-defined")
@@ -499,15 +501,15 @@ public interface EigenbaseNewResource {
 
   @BaseMessage("Personality does not maintain table''s row count in the catalog")
   @Property(name = "FeatureDefinition", value = "Eigenbase-defined")
-  ExInst<EigenbaseException> personalityManagesRowCount();
+  ExInst<CalciteException> personalityManagesRowCount();
 
   @BaseMessage("Personality does not support snapshot reads")
   @Property(name = "FeatureDefinition", value = "Eigenbase-defined")
-  ExInst<EigenbaseException> personalitySupportsSnapshots();
+  ExInst<CalciteException> personalitySupportsSnapshots();
 
   @BaseMessage("Personality does not support labels")
   @Property(name = "FeatureDefinition", value = "Eigenbase-defined")
-  ExInst<EigenbaseException> personalitySupportsLabels();
+  ExInst<CalciteException> personalitySupportsLabels();
 
   @BaseMessage("Require at least 1 argument")
   ExInst<SqlValidatorException> requireAtLeastOneArg();
@@ -534,10 +536,10 @@ public interface EigenbaseNewResource {
   ExInst<RuntimeException> firstParameterOfAdd(String className);
 
   @BaseMessage("FilterableTable.scan returned a filter that was not in the original list: {0}")
-  ExInst<EigenbaseException> filterableTableInventedFilter(String s);
+  ExInst<CalciteException> filterableTableInventedFilter(String s);
 
   @BaseMessage("FilterableTable.scan must not return null")
-  ExInst<EigenbaseException> filterableTableScanReturnedNull();
+  ExInst<CalciteException> filterableTableScanReturnedNull();
 }
 
-// End EigenbaseNewResource.java
+// End CalciteResource.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/runtime/Enumerables.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/runtime/Enumerables.java b/core/src/main/java/org/apache/calcite/runtime/Enumerables.java
index 92b14d6..c7e45f2 100644
--- a/core/src/main/java/org/apache/calcite/runtime/Enumerables.java
+++ b/core/src/main/java/org/apache/calcite/runtime/Enumerables.java
@@ -14,21 +14,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.runtime;
+package org.apache.calcite.runtime;
 
-import net.hydromatic.linq4j.AbstractEnumerable;
-import net.hydromatic.linq4j.Enumerable;
-import net.hydromatic.linq4j.Enumerator;
-import net.hydromatic.linq4j.function.EqualityComparer;
-import net.hydromatic.linq4j.function.Function1;
-import net.hydromatic.linq4j.function.Predicate1;
-
-import net.hydromatic.optiq.impl.interpreter.Row;
-
-import org.eigenbase.util.Bug;
+import org.apache.calcite.interpreter.Row;
+import org.apache.calcite.linq4j.AbstractEnumerable;
+import org.apache.calcite.linq4j.Enumerable;
+import org.apache.calcite.linq4j.Enumerator;
+import org.apache.calcite.linq4j.function.EqualityComparer;
+import org.apache.calcite.linq4j.function.Function1;
+import org.apache.calcite.linq4j.function.Predicate1;
+import org.apache.calcite.util.Bug;
 
 /**
- * Utilities for processing {@link net.hydromatic.linq4j.Enumerable}
+ * Utilities for processing {@link org.apache.calcite.linq4j.Enumerable}
  * collections.
  *
  * <p>This class is a place to put things not yet added to linq4j.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/runtime/EnumeratorCursor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/runtime/EnumeratorCursor.java b/core/src/main/java/org/apache/calcite/runtime/EnumeratorCursor.java
index 4ce8d1c..ec6795b 100644
--- a/core/src/main/java/org/apache/calcite/runtime/EnumeratorCursor.java
+++ b/core/src/main/java/org/apache/calcite/runtime/EnumeratorCursor.java
@@ -14,17 +14,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.runtime;
+package org.apache.calcite.runtime;
 
-import net.hydromatic.linq4j.Enumerator;
+import org.apache.calcite.linq4j.Enumerator;
 
 /**
- * Implementation of {@link net.hydromatic.avatica.Cursor} on top of an
- * {@link net.hydromatic.linq4j.Enumerator} that
+ * Implementation of {@link org.apache.calcite.avatica.Cursor} on top of an
+ * {@link org.apache.calcite.linq4j.Enumerator} that
  * returns a record for each row. The returned record is cached to avoid
  * multiple computations of current row.
  * For instance,
- * {@link net.hydromatic.optiq.rules.java.JavaRules.EnumerableCalcRel}
+ * {@link org.apache.calcite.adapter.enumerable.EnumerableCalc}
  * computes result just in {@code current()} method, thus it makes sense to
  * cache the result and make it available for all the accessors.
  *

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/runtime/Feature.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/runtime/Feature.java b/core/src/main/java/org/apache/calcite/runtime/Feature.java
index 288491d..f38df94 100644
--- a/core/src/main/java/org/apache/calcite/runtime/Feature.java
+++ b/core/src/main/java/org/apache/calcite/runtime/Feature.java
@@ -14,17 +14,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.resource;
+package org.apache.calcite.runtime;
 
 import java.lang.reflect.Method;
 import java.util.Locale;
 
-import org.eigenbase.util.EigenbaseContextException;
-
 /** SQL language feature. Expressed as the exception that would be thrown if it
  * were used while disabled. */
 public class Feature
-    extends Resources.ExInstWithCause<EigenbaseContextException> {
+    extends Resources.ExInstWithCause<CalciteContextException> {
   public Feature(String base, Locale locale, Method method, Object... args) {
     super(base, locale, method, args);
   }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/runtime/FlatLists.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/runtime/FlatLists.java b/core/src/main/java/org/apache/calcite/runtime/FlatLists.java
index 0956205..c2c887f 100644
--- a/core/src/main/java/org/apache/calcite/runtime/FlatLists.java
+++ b/core/src/main/java/org/apache/calcite/runtime/FlatLists.java
@@ -14,9 +14,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.runtime;
+package org.apache.calcite.runtime;
 
-import java.util.*;
+import java.util.AbstractList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.RandomAccess;
 
 /**
  * Space-efficient, comparable, immutable lists.
@@ -573,4 +580,3 @@ public class FlatLists {
 }
 
 // End FlatLists.java
-

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/runtime/Hook.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/runtime/Hook.java b/core/src/main/java/org/apache/calcite/runtime/Hook.java
index f8abae4..2d10b31 100644
--- a/core/src/main/java/org/apache/calcite/runtime/Hook.java
+++ b/core/src/main/java/org/apache/calcite/runtime/Hook.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.runtime;
+package org.apache.calcite.runtime;
 
 import com.google.common.base.Function;
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/runtime/Like.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/runtime/Like.java b/core/src/main/java/org/apache/calcite/runtime/Like.java
index a0b3c0e..91fa4d3 100644
--- a/core/src/main/java/org/apache/calcite/runtime/Like.java
+++ b/core/src/main/java/org/apache/calcite/runtime/Like.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.runtime;
+package org.apache.calcite.runtime;
 
 /**
  * Utilities for converting SQL {@code LIKE} and {@code SIMILAR} operators

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/runtime/ObjectEnumeratorCursor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/runtime/ObjectEnumeratorCursor.java b/core/src/main/java/org/apache/calcite/runtime/ObjectEnumeratorCursor.java
index 3f126ae..f73d7d6 100644
--- a/core/src/main/java/org/apache/calcite/runtime/ObjectEnumeratorCursor.java
+++ b/core/src/main/java/org/apache/calcite/runtime/ObjectEnumeratorCursor.java
@@ -14,13 +14,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.runtime;
+package org.apache.calcite.runtime;
 
-import net.hydromatic.linq4j.Enumerator;
+import org.apache.calcite.linq4j.Enumerator;
 
 /**
- * Implementation of {@link net.hydromatic.avatica.Cursor} on top of an
- * {@link net.hydromatic.linq4j.Enumerator} that
+ * Implementation of {@link org.apache.calcite.avatica.Cursor} on top of an
+ * {@link org.apache.calcite.linq4j.Enumerator} that
  * returns an {@link Object} for each row.
  */
 public class ObjectEnumeratorCursor extends AbstractCursor {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/runtime/RecordEnumeratorCursor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/runtime/RecordEnumeratorCursor.java b/core/src/main/java/org/apache/calcite/runtime/RecordEnumeratorCursor.java
index 46ce62f..0e922bd 100644
--- a/core/src/main/java/org/apache/calcite/runtime/RecordEnumeratorCursor.java
+++ b/core/src/main/java/org/apache/calcite/runtime/RecordEnumeratorCursor.java
@@ -14,15 +14,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.runtime;
+package org.apache.calcite.runtime;
 
-import net.hydromatic.linq4j.Enumerator;
+import org.apache.calcite.linq4j.Enumerator;
 
 import java.lang.reflect.Field;
 
 /**
- * Implementation of {@link net.hydromatic.avatica.Cursor} on top of an
- * {@link net.hydromatic.linq4j.Enumerator} that
+ * Implementation of {@link org.apache.calcite.avatica.Cursor} on top of an
+ * {@link org.apache.calcite.linq4j.Enumerator} that
  * returns a record for each row. The record is a synthetic class whose fields
  * are all public.
  *

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/runtime/ResultSetEnumerable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/runtime/ResultSetEnumerable.java b/core/src/main/java/org/apache/calcite/runtime/ResultSetEnumerable.java
index 3972004..b805a1c 100644
--- a/core/src/main/java/org/apache/calcite/runtime/ResultSetEnumerable.java
+++ b/core/src/main/java/org/apache/calcite/runtime/ResultSetEnumerable.java
@@ -14,16 +14,22 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.runtime;
+package org.apache.calcite.runtime;
 
-import net.hydromatic.linq4j.AbstractEnumerable;
-import net.hydromatic.linq4j.Enumerable;
-import net.hydromatic.linq4j.Enumerator;
-import net.hydromatic.linq4j.expressions.Primitive;
-import net.hydromatic.linq4j.function.Function0;
-import net.hydromatic.linq4j.function.Function1;
+import org.apache.calcite.linq4j.AbstractEnumerable;
+import org.apache.calcite.linq4j.Enumerable;
+import org.apache.calcite.linq4j.Enumerator;
+import org.apache.calcite.linq4j.function.Function0;
+import org.apache.calcite.linq4j.function.Function1;
+import org.apache.calcite.linq4j.tree.Primitive;
 
-import java.sql.*;
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+import java.sql.SQLFeatureNotSupportedException;
+import java.sql.Statement;
+import java.sql.Types;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.logging.Level;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/runtime/SortedMultiMap.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/runtime/SortedMultiMap.java b/core/src/main/java/org/apache/calcite/runtime/SortedMultiMap.java
index 0aabb83..d4fa04e 100644
--- a/core/src/main/java/org/apache/calcite/runtime/SortedMultiMap.java
+++ b/core/src/main/java/org/apache/calcite/runtime/SortedMultiMap.java
@@ -14,9 +14,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.runtime;
+package org.apache.calcite.runtime;
 
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
 
 /**
  * Map that allows you to partition values into lists according to a common

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/runtime/Spacer.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/runtime/Spacer.java b/core/src/main/java/org/apache/calcite/runtime/Spacer.java
index c5b145c..e028716 100644
--- a/core/src/main/java/org/apache/calcite/runtime/Spacer.java
+++ b/core/src/main/java/org/apache/calcite/runtime/Spacer.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.runtime;
+package org.apache.calcite.runtime;
 
 /**
  * Efficiently writes strings of spaces.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/runtime/Spaces.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/runtime/Spaces.java b/core/src/main/java/org/apache/calcite/runtime/Spaces.java
index c34e942..9921579 100644
--- a/core/src/main/java/org/apache/calcite/runtime/Spaces.java
+++ b/core/src/main/java/org/apache/calcite/runtime/Spaces.java
@@ -14,10 +14,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.runtime;
-
-import java.io.*;
-import java.util.*;
+package org.apache.calcite.runtime;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.util.AbstractList;
+import java.util.Arrays;
+import java.util.List;
 import java.util.concurrent.CopyOnWriteArrayList;
 
 /** Utilities for creating strings of spaces. */

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java b/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
index 9b4faac..a213ac3 100644
--- a/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
+++ b/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
@@ -14,26 +14,28 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.runtime;
-
-import net.hydromatic.avatica.ByteString;
-
-import net.hydromatic.linq4j.Enumerable;
-import net.hydromatic.linq4j.Linq4j;
-import net.hydromatic.linq4j.expressions.Primitive;
-import net.hydromatic.linq4j.function.Deterministic;
-import net.hydromatic.linq4j.function.Function1;
-import net.hydromatic.linq4j.function.NonDeterministic;
-
-import net.hydromatic.optiq.DataContext;
-
-import org.eigenbase.util14.DateTimeUtil;
-
-import java.math.*;
+package org.apache.calcite.runtime;
+
+import org.apache.calcite.DataContext;
+import org.apache.calcite.avatica.ByteString;
+import org.apache.calcite.linq4j.Enumerable;
+import org.apache.calcite.linq4j.Linq4j;
+import org.apache.calcite.linq4j.function.Deterministic;
+import org.apache.calcite.linq4j.function.Function1;
+import org.apache.calcite.linq4j.function.NonDeterministic;
+import org.apache.calcite.linq4j.tree.Primitive;
+import org.apache.calcite.util.DateTimeUtil;
+
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.math.MathContext;
 import java.sql.SQLException;
 import java.sql.Timestamp;
 import java.text.DecimalFormat;
-import java.util.*;
+import java.util.Date;
+import java.util.List;
+import java.util.Map;
+import java.util.TimeZone;
 import java.util.regex.Pattern;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/runtime/Typed.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/runtime/Typed.java b/core/src/main/java/org/apache/calcite/runtime/Typed.java
index 1e2fc78..8620981 100644
--- a/core/src/main/java/org/apache/calcite/runtime/Typed.java
+++ b/core/src/main/java/org/apache/calcite/runtime/Typed.java
@@ -14,12 +14,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.runtime;
+package org.apache.calcite.runtime;
 
 import java.lang.reflect.Type;
 
 /**
- * Adds type information to a {@link net.hydromatic.linq4j.Enumerable}.
+ * Adds type information to a {@link org.apache.calcite.linq4j.Enumerable}.
  */
 public interface Typed {
   /**

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/runtime/Unit.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/runtime/Unit.java b/core/src/main/java/org/apache/calcite/runtime/Unit.java
index 08cee49..c3e29ca 100644
--- a/core/src/main/java/org/apache/calcite/runtime/Unit.java
+++ b/core/src/main/java/org/apache/calcite/runtime/Unit.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.runtime;
+package org.apache.calcite.runtime;
 
 /**
  * Synthetic record with zero fields.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/runtime/Utilities.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/runtime/Utilities.java b/core/src/main/java/org/apache/calcite/runtime/Utilities.java
index 73e7d4d..691afd5 100644
--- a/core/src/main/java/org/apache/calcite/runtime/Utilities.java
+++ b/core/src/main/java/org/apache/calcite/runtime/Utilities.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.runtime;
+package org.apache.calcite.runtime;
 
 import java.util.Iterator;
 import java.util.List;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/runtime/package-info.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/runtime/package-info.java b/core/src/main/java/org/apache/calcite/runtime/package-info.java
index 14a6035..25e404f 100644
--- a/core/src/main/java/org/apache/calcite/runtime/package-info.java
+++ b/core/src/main/java/org/apache/calcite/runtime/package-info.java
@@ -18,6 +18,6 @@
 /**
  * Utilities required at runtime.
  */
-package net.hydromatic.optiq.runtime;
+package org.apache.calcite.runtime;
 
 // End package-info.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/schema/AggregateFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/schema/AggregateFunction.java b/core/src/main/java/org/apache/calcite/schema/AggregateFunction.java
index 274ed73..30e6725 100644
--- a/core/src/main/java/org/apache/calcite/schema/AggregateFunction.java
+++ b/core/src/main/java/org/apache/calcite/schema/AggregateFunction.java
@@ -14,10 +14,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq;
+package org.apache.calcite.schema;
 
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.reltype.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
 
 /**
  * Function that combines several values into a scalar result.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/schema/FilterableTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/schema/FilterableTable.java b/core/src/main/java/org/apache/calcite/schema/FilterableTable.java
index 7bc3a26..753fe91 100644
--- a/core/src/main/java/org/apache/calcite/schema/FilterableTable.java
+++ b/core/src/main/java/org/apache/calcite/schema/FilterableTable.java
@@ -14,11 +14,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq;
+package org.apache.calcite.schema;
 
-import net.hydromatic.linq4j.Enumerable;
-
-import org.eigenbase.rex.RexNode;
+import org.apache.calcite.DataContext;
+import org.apache.calcite.linq4j.Enumerable;
+import org.apache.calcite.rex.RexNode;
 
 import java.util.List;
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/schema/Function.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/schema/Function.java b/core/src/main/java/org/apache/calcite/schema/Function.java
index da4b3b2..cdca7a5 100644
--- a/core/src/main/java/org/apache/calcite/schema/Function.java
+++ b/core/src/main/java/org/apache/calcite/schema/Function.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq;
+package org.apache.calcite.schema;
 
 import java.util.List;
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/schema/FunctionParameter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/schema/FunctionParameter.java b/core/src/main/java/org/apache/calcite/schema/FunctionParameter.java
index fe25b5d..11e50c4 100644
--- a/core/src/main/java/org/apache/calcite/schema/FunctionParameter.java
+++ b/core/src/main/java/org/apache/calcite/schema/FunctionParameter.java
@@ -14,10 +14,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq;
+package org.apache.calcite.schema;
 
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.reltype.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
 
 /**
  * Parameter to a {@link Function}.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/schema/ImplementableAggFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/schema/ImplementableAggFunction.java b/core/src/main/java/org/apache/calcite/schema/ImplementableAggFunction.java
index 277b1d1..94b0b45 100644
--- a/core/src/main/java/org/apache/calcite/schema/ImplementableAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/schema/ImplementableAggFunction.java
@@ -14,17 +14,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq;
+package org.apache.calcite.schema;
 
-import net.hydromatic.optiq.rules.java.AggImplementor;
+import org.apache.calcite.adapter.enumerable.AggImplementor;
 
 /**
  * Function that can be translated to java code.
  *
- * @see net.hydromatic.optiq.rules.java.AggImplementor
- * @see net.hydromatic.optiq.rules.java.WinAggImplementor
- * @see net.hydromatic.optiq.rules.java.StrictAggImplementor
- * @see net.hydromatic.optiq.rules.java.StrictWinAggImplementor
+ * @see org.apache.calcite.adapter.enumerable.AggImplementor
+ * @see org.apache.calcite.adapter.enumerable.WinAggImplementor
+ * @see org.apache.calcite.adapter.enumerable.StrictAggImplementor
+ * @see org.apache.calcite.adapter.enumerable.StrictWinAggImplementor
  */
 public interface ImplementableAggFunction extends AggregateFunction {
   /**

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/schema/ImplementableFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/schema/ImplementableFunction.java b/core/src/main/java/org/apache/calcite/schema/ImplementableFunction.java
index 2660efa..e138c3e 100644
--- a/core/src/main/java/org/apache/calcite/schema/ImplementableFunction.java
+++ b/core/src/main/java/org/apache/calcite/schema/ImplementableFunction.java
@@ -14,15 +14,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq;
+package org.apache.calcite.schema;
 
-import net.hydromatic.optiq.rules.java.CallImplementor;
+import org.apache.calcite.adapter.enumerable.CallImplementor;
 
 /**
  * Function that can be translated to java code.
- * <p>
- * @see net.hydromatic.optiq.ScalarFunction
- * @see net.hydromatic.optiq.TableFunction
+ *
+ * @see ScalarFunction
+ * @see TableFunction
  */
 public interface ImplementableFunction extends Function {
   /**

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/schema/Member.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/schema/Member.java b/core/src/main/java/org/apache/calcite/schema/Member.java
index 88984d3..e2eee27 100644
--- a/core/src/main/java/org/apache/calcite/schema/Member.java
+++ b/core/src/main/java/org/apache/calcite/schema/Member.java
@@ -14,11 +14,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq;
+package org.apache.calcite.schema;
 
-import net.hydromatic.linq4j.Queryable;
-
-import org.eigenbase.reltype.RelDataType;
+import org.apache.calcite.linq4j.Queryable;
+import org.apache.calcite.rel.type.RelDataType;
 
 import java.util.List;
 
@@ -69,10 +68,10 @@ public interface Member {
 
   /**
    * Evaluates this member to yield a result. The result is a
-   * {@link net.hydromatic.linq4j.Queryable}.
+   * {@link org.apache.calcite.linq4j.Queryable}.
    *
    * @param schemaInstance Object that is an instance of the containing
-   *                       {@link net.hydromatic.optiq.Schema}
+   *                       {@link Schema}
    * @param arguments List of arguments to the call; must match
    *                  {@link #getParameters() parameters} in number and type
    *

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/schema/ModifiableTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/schema/ModifiableTable.java b/core/src/main/java/org/apache/calcite/schema/ModifiableTable.java
index 527cbea..85b2a87 100644
--- a/core/src/main/java/org/apache/calcite/schema/ModifiableTable.java
+++ b/core/src/main/java/org/apache/calcite/schema/ModifiableTable.java
@@ -14,13 +14,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq;
+package org.apache.calcite.schema;
 
-import net.hydromatic.optiq.prepare.Prepare;
-
-import org.eigenbase.rel.RelNode;
-import org.eigenbase.rel.TableModificationRelBase;
-import org.eigenbase.relopt.*;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.prepare.Prepare;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.TableModify;
 
 import java.util.Collection;
 import java.util.List;
@@ -37,12 +37,12 @@ public interface ModifiableTable extends QueryableTable {
   Collection getModifiableCollection();
 
   /** Creates a relational expression that modifies this table. */
-  TableModificationRelBase toModificationRel(
+  TableModify toModificationRel(
       RelOptCluster cluster,
       RelOptTable table,
       Prepare.CatalogReader catalogReader,
       RelNode child,
-      TableModificationRelBase.Operation operation,
+      TableModify.Operation operation,
       List<String> updateColumnList,
       boolean flattened);
 }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/schema/ProjectableFilterableTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/schema/ProjectableFilterableTable.java b/core/src/main/java/org/apache/calcite/schema/ProjectableFilterableTable.java
index e713b53..2f56865 100644
--- a/core/src/main/java/org/apache/calcite/schema/ProjectableFilterableTable.java
+++ b/core/src/main/java/org/apache/calcite/schema/ProjectableFilterableTable.java
@@ -14,11 +14,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq;
+package org.apache.calcite.schema;
 
-import net.hydromatic.linq4j.Enumerable;
-
-import org.eigenbase.rex.RexNode;
+import org.apache.calcite.DataContext;
+import org.apache.calcite.linq4j.Enumerable;
+import org.apache.calcite.rex.RexNode;
 
 import java.util.List;
 
@@ -30,8 +30,8 @@ import java.util.List;
  * <p>If you wish to write a table that can apply projects but not filters,
  * simply decline all filters.</p>
  *
- * @see net.hydromatic.optiq.ScannableTable
- * @see net.hydromatic.optiq.FilterableTable
+ * @see ScannableTable
+ * @see FilterableTable
  */
 public interface ProjectableFilterableTable extends Table {
   /** Returns an enumerable over the rows in this Table.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/schema/QueryableTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/schema/QueryableTable.java b/core/src/main/java/org/apache/calcite/schema/QueryableTable.java
index a7c57ae..ab831cd 100644
--- a/core/src/main/java/org/apache/calcite/schema/QueryableTable.java
+++ b/core/src/main/java/org/apache/calcite/schema/QueryableTable.java
@@ -14,11 +14,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq;
+package org.apache.calcite.schema;
 
-import net.hydromatic.linq4j.QueryProvider;
-import net.hydromatic.linq4j.Queryable;
-import net.hydromatic.linq4j.expressions.Expression;
+import org.apache.calcite.linq4j.QueryProvider;
+import org.apache.calcite.linq4j.Queryable;
+import org.apache.calcite.linq4j.tree.Expression;
 
 import java.lang.reflect.Type;
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/schema/ScalarFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/schema/ScalarFunction.java b/core/src/main/java/org/apache/calcite/schema/ScalarFunction.java
index 9d8423d..13e7a7c 100644
--- a/core/src/main/java/org/apache/calcite/schema/ScalarFunction.java
+++ b/core/src/main/java/org/apache/calcite/schema/ScalarFunction.java
@@ -14,10 +14,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq;
+package org.apache.calcite.schema;
 
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.reltype.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
 
 /**
  * Function that returns a scalar result.


[49/58] [abbrv] [partial] incubator-calcite git commit: [CALCITE-306] Standardize code style for "import package.*; "

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableCalc.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableCalc.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableCalc.java
new file mode 100644
index 0000000..69afcb4
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableCalc.java
@@ -0,0 +1,221 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.linq4j.Enumerator;
+import org.apache.calcite.linq4j.tree.BlockBuilder;
+import org.apache.calcite.linq4j.tree.BlockStatement;
+import org.apache.calcite.linq4j.tree.Blocks;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.linq4j.tree.MemberDeclaration;
+import org.apache.calcite.linq4j.tree.ParameterExpression;
+import org.apache.calcite.linq4j.tree.Types;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Calc;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.util.BuiltInMethod;
+import org.apache.calcite.util.Pair;
+
+import com.google.common.collect.ImmutableList;
+
+import java.lang.reflect.Modifier;
+import java.lang.reflect.Type;
+import java.util.Collections;
+import java.util.List;
+
+import static org.apache.calcite.adapter.enumerable.EnumUtils.BRIDGE_METHODS;
+import static org.apache.calcite.adapter.enumerable.EnumUtils.NO_EXPRS;
+import static org.apache.calcite.adapter.enumerable.EnumUtils.NO_PARAMS;
+
+/** Implementation of {@link org.apache.calcite.rel.core.Calc} in
+ * {@link org.apache.calcite.adapter.enumerable.EnumerableConvention enumerable calling convention}. */
+public class EnumerableCalc extends Calc implements EnumerableRel {
+  public EnumerableCalc(
+      RelOptCluster cluster,
+      RelTraitSet traitSet,
+      RelNode child,
+      RelDataType rowType,
+      RexProgram program,
+      List<RelCollation> collationList) {
+    super(cluster, traitSet, child, rowType, program, collationList);
+    assert getConvention() instanceof EnumerableConvention;
+    assert !program.containsAggs();
+  }
+
+  @Override public EnumerableCalc copy(RelTraitSet traitSet, RelNode child,
+      RexProgram program, List<RelCollation> collationList) {
+    // we do not need to copy program; it is immutable
+    return new EnumerableCalc(getCluster(), traitSet, child,
+        program.getOutputRowType(), program, collationList);
+  }
+
+  public Result implement(EnumerableRelImplementor implementor, Prefer pref) {
+    final JavaTypeFactory typeFactory = implementor.getTypeFactory();
+    final BlockBuilder builder = new BlockBuilder();
+    final EnumerableRel child = (EnumerableRel) getInput();
+
+    final Result result =
+        implementor.visitChild(this, 0, child, pref);
+
+    final PhysType physType =
+        PhysTypeImpl.of(
+            typeFactory, getRowType(), pref.prefer(result.format));
+
+    // final Enumerable<Employee> inputEnumerable = <<child adapter>>;
+    // return new Enumerable<IntString>() {
+    //     Enumerator<IntString> enumerator() {
+    //         return new Enumerator<IntString>() {
+    //             public void reset() {
+    // ...
+    Type outputJavaType = physType.getJavaRowType();
+    final Type enumeratorType =
+        Types.of(
+            Enumerator.class, outputJavaType);
+    Type inputJavaType = result.physType.getJavaRowType();
+    ParameterExpression inputEnumerator =
+        Expressions.parameter(
+            Types.of(
+                Enumerator.class, inputJavaType),
+            "inputEnumerator");
+    Expression input =
+        RexToLixTranslator.convert(
+            Expressions.call(
+                inputEnumerator,
+                BuiltInMethod.ENUMERATOR_CURRENT.method),
+            inputJavaType);
+
+    BlockStatement moveNextBody;
+    if (program.getCondition() == null) {
+      moveNextBody =
+          Blocks.toFunctionBlock(
+              Expressions.call(
+                  inputEnumerator,
+                  BuiltInMethod.ENUMERATOR_MOVE_NEXT.method));
+    } else {
+      final BlockBuilder builder2 = new BlockBuilder();
+      Expression condition =
+          RexToLixTranslator.translateCondition(
+              program,
+              typeFactory,
+              builder2,
+              new RexToLixTranslator.InputGetterImpl(
+                  Collections.singletonList(
+                      Pair.of(input, result.physType))));
+      builder2.add(
+          Expressions.ifThen(
+              condition,
+              Expressions.return_(
+                  null, Expressions.constant(true))));
+      moveNextBody =
+          Expressions.block(
+              Expressions.while_(
+                  Expressions.call(
+                      inputEnumerator,
+                      BuiltInMethod.ENUMERATOR_MOVE_NEXT.method),
+                  builder2.toBlock()),
+              Expressions.return_(
+                  null,
+                  Expressions.constant(false)));
+    }
+
+    final BlockBuilder builder3 = new BlockBuilder();
+    List<Expression> expressions =
+        RexToLixTranslator.translateProjects(
+            program,
+            typeFactory,
+            builder3,
+            physType,
+            new RexToLixTranslator.InputGetterImpl(
+                Collections.singletonList(
+                    Pair.of(input, result.physType))));
+    builder3.add(
+        Expressions.return_(
+            null, physType.record(expressions)));
+    BlockStatement currentBody =
+        builder3.toBlock();
+
+    final Expression inputEnumerable =
+        builder.append(
+            "inputEnumerable", result.block, false);
+    final Expression body =
+        Expressions.new_(
+            enumeratorType,
+            NO_EXPRS,
+            Expressions.<MemberDeclaration>list(
+                Expressions.fieldDecl(
+                    Modifier.PUBLIC
+                    | Modifier.FINAL,
+                    inputEnumerator,
+                    Expressions.call(
+                        inputEnumerable,
+                        BuiltInMethod.ENUMERABLE_ENUMERATOR.method)),
+                EnumUtils.overridingMethodDecl(
+                    BuiltInMethod.ENUMERATOR_RESET.method,
+                    NO_PARAMS,
+                    Blocks.toFunctionBlock(
+                        Expressions.call(
+                            inputEnumerator,
+                            BuiltInMethod.ENUMERATOR_RESET.method))),
+                EnumUtils.overridingMethodDecl(
+                    BuiltInMethod.ENUMERATOR_MOVE_NEXT.method,
+                    NO_PARAMS,
+                    moveNextBody),
+                EnumUtils.overridingMethodDecl(
+                    BuiltInMethod.ENUMERATOR_CLOSE.method,
+                    NO_PARAMS,
+                    Blocks.toFunctionBlock(
+                        Expressions.call(
+                            inputEnumerator,
+                            BuiltInMethod.ENUMERATOR_CLOSE.method))),
+                Expressions.methodDecl(
+                    Modifier.PUBLIC,
+                    BRIDGE_METHODS
+                        ? Object.class
+                        : outputJavaType,
+                    "current",
+                    NO_PARAMS,
+                    currentBody)));
+    builder.add(
+        Expressions.return_(
+            null,
+            Expressions.new_(
+                BuiltInMethod.ABSTRACT_ENUMERABLE_CTOR.constructor,
+                // TODO: generics
+                //   Collections.singletonList(inputRowType),
+                NO_EXPRS,
+                ImmutableList.<MemberDeclaration>of(
+                    Expressions.methodDecl(
+                        Modifier.PUBLIC,
+                        enumeratorType,
+                        BuiltInMethod.ENUMERABLE_ENUMERATOR.method.getName(),
+                        NO_PARAMS,
+                        Blocks.toFunctionBlock(body))))));
+    return implementor.result(physType, builder.toBlock());
+  }
+
+  public RexProgram getProgram() {
+    return program;
+  }
+}
+
+// End EnumerableCalc.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableCalcRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableCalcRule.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableCalcRule.java
new file mode 100644
index 0000000..05d907a
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableCalcRule.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.logical.LogicalCalc;
+import org.apache.calcite.rex.RexMultisetUtil;
+import org.apache.calcite.rex.RexProgram;
+
+/**
+ * Rule to convert a {@link org.apache.calcite.rel.logical.LogicalCalc} to an
+ * {@link EnumerableCalc}.
+ */
+class EnumerableCalcRule extends ConverterRule {
+  EnumerableCalcRule() {
+    super(LogicalCalc.class, Convention.NONE, EnumerableConvention.INSTANCE,
+        "EnumerableCalcRule");
+  }
+
+  public RelNode convert(RelNode rel) {
+    final LogicalCalc calc = (LogicalCalc) rel;
+
+    // If there's a multiset, let FarragoMultisetSplitter work on it
+    // first.
+    final RexProgram program = calc.getProgram();
+    if (EnumUtils.B
+        && RexMultisetUtil.containsMultiset(program)
+        || program.containsAggs()) {
+      return null;
+    }
+
+    return new EnumerableCalc(
+        rel.getCluster(),
+        rel.getTraitSet().replace(EnumerableConvention.INSTANCE),
+        convert(
+            calc.getInput(),
+            calc.getInput().getTraitSet()
+                .replace(EnumerableConvention.INSTANCE)),
+        calc.getRowType(),
+        program,
+        calc.getCollationList());
+  }
+}
+
+// End EnumerableCalcRule.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableCollect.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableCollect.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableCollect.java
new file mode 100644
index 0000000..99c2a1a
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableCollect.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.linq4j.tree.BlockBuilder;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Collect;
+import org.apache.calcite.util.BuiltInMethod;
+
+/** Implementation of {@link org.apache.calcite.rel.core.Collect} in
+ * {@link org.apache.calcite.adapter.enumerable.EnumerableConvention enumerable calling convention}. */
+public class EnumerableCollect extends Collect implements EnumerableRel {
+  public EnumerableCollect(RelOptCluster cluster, RelTraitSet traitSet,
+      RelNode child, String fieldName) {
+    super(cluster, traitSet, child, fieldName);
+    assert getConvention() instanceof EnumerableConvention;
+    assert getConvention() == child.getConvention();
+  }
+
+  @Override public EnumerableCollect copy(RelTraitSet traitSet,
+      RelNode newInput) {
+    return new EnumerableCollect(getCluster(), traitSet, newInput, fieldName);
+  }
+
+  public Result implement(EnumerableRelImplementor implementor, Prefer pref) {
+    final BlockBuilder builder = new BlockBuilder();
+    final EnumerableRel child = (EnumerableRel) getInput();
+    final Result result = implementor.visitChild(this, 0, child, pref);
+    final PhysType physType =
+        PhysTypeImpl.of(
+            implementor.getTypeFactory(),
+            getRowType(),
+            result.format);
+
+    // final Enumerable<Employee> child = <<child adapter>>;
+    // final List<Employee> list = child.toList();
+    Expression child_ =
+        builder.append(
+            "child", result.block);
+    Expression list_ =
+        builder.append("list",
+            Expressions.call(child_,
+                BuiltInMethod.ENUMERABLE_TO_LIST.method));
+
+    builder.add(
+        Expressions.return_(null,
+            Expressions.call(
+                BuiltInMethod.SINGLETON_ENUMERABLE.method, list_)));
+    return implementor.result(physType, builder.toBlock());
+  }
+}
+
+// End EnumerableCollect.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableCollectRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableCollectRule.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableCollectRule.java
new file mode 100644
index 0000000..4ab49b4
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableCollectRule.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.core.Collect;
+
+/**
+ * Rule to convert an {@link org.apache.calcite.rel.core.Collect} to an
+ * {@link EnumerableCollect}.
+ */
+class EnumerableCollectRule extends ConverterRule {
+  EnumerableCollectRule() {
+    super(Collect.class, Convention.NONE, EnumerableConvention.INSTANCE,
+        "EnumerableCollectRule");
+  }
+
+  public RelNode convert(RelNode rel) {
+    final Collect collect = (Collect) rel;
+    final RelTraitSet traitSet =
+        collect.getTraitSet().replace(EnumerableConvention.INSTANCE);
+    final RelNode input = collect.getInput();
+    return new EnumerableCollect(
+        rel.getCluster(),
+        traitSet,
+        convert(input,
+            input.getTraitSet().replace(EnumerableConvention.INSTANCE)),
+        collect.getFieldName());
+  }
+}
+
+// End EnumerableCollectRule.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableConvention.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableConvention.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableConvention.java
index 7a5cceb..a8ad841 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableConvention.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableConvention.java
@@ -14,19 +14,22 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.rules.java;
+package org.apache.calcite.adapter.enumerable;
 
-import org.eigenbase.relopt.*;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.ConventionTraitDef;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTrait;
+import org.apache.calcite.plan.RelTraitDef;
 
 /**
  * Family of calling conventions that return results as an
- * {@link net.hydromatic.linq4j.Enumerable}.
+ * {@link org.apache.calcite.linq4j.Enumerable}.
  */
 public enum EnumerableConvention implements Convention {
   INSTANCE;
 
-  @Override
-  public String toString() {
+  @Override public String toString() {
     return getName();
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableEmptyRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableEmptyRule.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableEmptyRule.java
new file mode 100644
index 0000000..8f57d84
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableEmptyRule.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.core.Empty;
+import org.apache.calcite.rex.RexLiteral;
+
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
+/** Planner rule that converts an {@link org.apache.calcite.rel.core.Empty}
+ * relational expression
+ * {@link org.apache.calcite.adapter.enumerable.EnumerableConvention enumerable calling convention}. */
+public class EnumerableEmptyRule extends ConverterRule {
+  EnumerableEmptyRule() {
+    super(Empty.class, Convention.NONE, EnumerableConvention.INSTANCE,
+        "EnumerableEmptyRule");
+  }
+
+  public RelNode convert(RelNode rel) {
+    Empty empty = (Empty) rel;
+    return new EnumerableValues(
+        empty.getCluster(),
+        empty.getRowType(),
+        ImmutableList.<List<RexLiteral>>of(),
+        empty.getTraitSet().replace(EnumerableConvention.INSTANCE));
+  }
+}
+
+// End EnumerableEmptyRule.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableFilter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableFilter.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableFilter.java
new file mode 100644
index 0000000..ae68216
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableFilter.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rex.RexNode;
+
+/** Implementation of {@link org.apache.calcite.rel.core.Filter} in
+ * {@link org.apache.calcite.adapter.enumerable.EnumerableConvention enumerable calling convention}. */
+public class EnumerableFilter
+    extends Filter
+    implements EnumerableRel {
+  public EnumerableFilter(
+      RelOptCluster cluster,
+      RelTraitSet traitSet,
+      RelNode child,
+      RexNode condition) {
+    super(cluster, traitSet, child, condition);
+    assert getConvention() instanceof EnumerableConvention;
+  }
+
+  public EnumerableFilter copy(RelTraitSet traitSet, RelNode input,
+      RexNode condition) {
+    return new EnumerableFilter(getCluster(), traitSet, input, condition);
+  }
+
+  public Result implement(EnumerableRelImplementor implementor, Prefer pref) {
+    // EnumerableCalc is always better
+    throw new UnsupportedOperationException();
+  }
+}
+
+// End EnumerableFilter.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableFilterRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableFilterRule.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableFilterRule.java
new file mode 100644
index 0000000..81bbaf2
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableFilterRule.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.logical.LogicalFilter;
+import org.apache.calcite.rex.RexMultisetUtil;
+import org.apache.calcite.rex.RexOver;
+
+/**
+ * Rule to convert a {@link org.apache.calcite.rel.logical.LogicalFilter} to an
+ * {@link EnumerableFilter}.
+ */
+class EnumerableFilterRule extends ConverterRule {
+  EnumerableFilterRule() {
+    super(LogicalFilter.class, Convention.NONE, EnumerableConvention.INSTANCE,
+        "EnumerableFilterRule");
+  }
+
+  public RelNode convert(RelNode rel) {
+    final LogicalFilter filter = (LogicalFilter) rel;
+
+    if (EnumUtils.B
+        && RexMultisetUtil.containsMultiset(filter.getCondition(), true)
+        || RexOver.containsOver(filter.getCondition())) {
+      return null;
+    }
+
+    return new EnumerableFilter(
+        rel.getCluster(),
+        rel.getTraitSet().replace(EnumerableConvention.INSTANCE),
+        convert(
+            filter.getInput(),
+            filter.getInput().getTraitSet()
+                .replace(EnumerableConvention.INSTANCE)),
+        filter.getCondition());
+  }
+}
+
+// End EnumerableFilterRule.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableFilterToCalcRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableFilterToCalcRule.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableFilterToCalcRule.java
new file mode 100644
index 0000000..901dc6b
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableFilterToCalcRule.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexProgramBuilder;
+
+import com.google.common.collect.ImmutableList;
+
+/** Variant of {@link org.apache.calcite.rel.rules.FilterToCalcRule} for
+ * {@link org.apache.calcite.adapter.enumerable.EnumerableConvention enumerable calling convention}. */
+public class EnumerableFilterToCalcRule extends RelOptRule {
+  EnumerableFilterToCalcRule() {
+    super(operand(EnumerableFilter.class, any()));
+  }
+
+  public void onMatch(RelOptRuleCall call) {
+    final EnumerableFilter filter = call.rel(0);
+    final RelNode rel = filter.getInput();
+
+    // Create a program containing a filter.
+    final RexBuilder rexBuilder = filter.getCluster().getRexBuilder();
+    final RelDataType inputRowType = rel.getRowType();
+    final RexProgramBuilder programBuilder =
+        new RexProgramBuilder(inputRowType, rexBuilder);
+    programBuilder.addIdentity();
+    programBuilder.addCondition(filter.getCondition());
+    final RexProgram program = programBuilder.getProgram();
+
+    final EnumerableCalc calc =
+        new EnumerableCalc(
+            filter.getCluster(),
+            filter.getTraitSet(),
+            rel,
+            inputRowType,
+            program,
+            ImmutableList.<RelCollation>of());
+    call.transformTo(calc);
+  }
+}
+
+// End EnumerableFilterToCalcRule.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableInterpreter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableInterpreter.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableInterpreter.java
new file mode 100644
index 0000000..64b0c6f
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableInterpreter.java
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.interpreter.Interpreter;
+import org.apache.calcite.linq4j.tree.BlockBuilder;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.SingleRel;
+import org.apache.calcite.util.BuiltInMethod;
+
+import java.util.List;
+
+/** Relational expression that executes its children using an interpreter.
+ *
+ * <p>Although quite a few kinds of {@link org.apache.calcite.rel.RelNode} can
+ * be interpreted, this is only created by default for
+ * {@link org.apache.calcite.schema.FilterableTable} and
+ * {@link org.apache.calcite.schema.ProjectableFilterableTable}.
+ */
+public class EnumerableInterpreter extends SingleRel
+    implements EnumerableRel {
+  private final double factor;
+
+  /** Creates an EnumerableInterpreterRel. */
+  public EnumerableInterpreter(RelOptCluster cluster,
+      RelTraitSet traitSet, RelNode input, double factor) {
+    super(cluster, traitSet, input);
+    this.factor = factor;
+  }
+
+  @Override public RelOptCost computeSelfCost(RelOptPlanner planner) {
+    return super.computeSelfCost(planner).multiplyBy(factor);
+  }
+
+  @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+    return new EnumerableInterpreter(getCluster(), traitSet, sole(inputs),
+        factor);
+  }
+
+  public Result implement(EnumerableRelImplementor implementor, Prefer pref) {
+    final JavaTypeFactory typeFactory = implementor.getTypeFactory();
+    final BlockBuilder builder = new BlockBuilder();
+    final PhysType physType =
+        PhysTypeImpl.of(typeFactory, getRowType(), JavaRowFormat.ARRAY);
+    final Expression interpreter_ = builder.append("interpreter",
+        Expressions.new_(Interpreter.class,
+            implementor.getRootExpression(),
+            implementor.stash(getInput(), RelNode.class)));
+    final Expression sliced_ =
+        getRowType().getFieldCount() == 1
+            ? Expressions.call(BuiltInMethod.SLICE0.method, interpreter_)
+            : interpreter_;
+    builder.add(sliced_);
+    return implementor.result(physType, builder.toBlock());
+  }
+}
+
+// End EnumerableInterpreter.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableIntersect.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableIntersect.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableIntersect.java
new file mode 100644
index 0000000..6c74c17
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableIntersect.java
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.linq4j.tree.BlockBuilder;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Intersect;
+import org.apache.calcite.util.BuiltInMethod;
+
+import java.util.List;
+
+/** Implementation of {@link org.apache.calcite.rel.core.Intersect} in
+ * {@link org.apache.calcite.adapter.enumerable.EnumerableConvention enumerable calling convention}. */
+public class EnumerableIntersect extends Intersect implements EnumerableRel {
+  public EnumerableIntersect(RelOptCluster cluster, RelTraitSet traitSet,
+      List<RelNode> inputs, boolean all) {
+    super(cluster, traitSet, inputs, all);
+    assert !all;
+  }
+
+  public EnumerableIntersect copy(RelTraitSet traitSet, List<RelNode> inputs,
+      boolean all) {
+    return new EnumerableIntersect(getCluster(), traitSet, inputs, all);
+  }
+
+  public Result implement(EnumerableRelImplementor implementor, Prefer pref) {
+    final BlockBuilder builder = new BlockBuilder();
+    Expression intersectExp = null;
+    for (Ord<RelNode> ord : Ord.zip(inputs)) {
+      EnumerableRel input = (EnumerableRel) ord.e;
+      final Result result = implementor.visitChild(this, ord.i, input, pref);
+      Expression childExp =
+          builder.append(
+              "child" + ord.i,
+              result.block);
+
+      if (intersectExp == null) {
+        intersectExp = childExp;
+      } else {
+        intersectExp =
+            Expressions.call(
+                intersectExp,
+                all
+                    ? BuiltInMethod.CONCAT.method
+                    : BuiltInMethod.INTERSECT.method,
+                childExp);
+      }
+
+      // Once the first input has chosen its format, ask for the same for
+      // other inputs.
+      pref = pref.of(result.format);
+    }
+
+    builder.add(intersectExp);
+    final PhysType physType =
+        PhysTypeImpl.of(
+            implementor.getTypeFactory(),
+            getRowType(),
+            pref.prefer(JavaRowFormat.CUSTOM));
+    return implementor.result(physType, builder.toBlock());
+  }
+}
+
+// End EnumerableIntersect.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableIntersectRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableIntersectRule.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableIntersectRule.java
new file mode 100644
index 0000000..f906d47
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableIntersectRule.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.logical.LogicalIntersect;
+
+/**
+ * Rule to convert a
+ * {@link org.apache.calcite.rel.logical.LogicalIntersect} to an
+ * {@link EnumerableIntersect}.
+ */
+class EnumerableIntersectRule extends ConverterRule {
+  EnumerableIntersectRule() {
+    super(LogicalIntersect.class, Convention.NONE,
+        EnumerableConvention.INSTANCE, "EnumerableIntersectRule");
+  }
+
+  public RelNode convert(RelNode rel) {
+    final LogicalIntersect intersect = (LogicalIntersect) rel;
+    if (intersect.all) {
+      return null; // INTERSECT ALL not implemented
+    }
+    final EnumerableConvention out = EnumerableConvention.INSTANCE;
+    final RelTraitSet traitSet = intersect.getTraitSet().replace(out);
+    return new EnumerableIntersect(rel.getCluster(), traitSet,
+        convertList(intersect.getInputs(), out), false);
+  }
+}
+
+// End EnumerableIntersectRule.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableJoin.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableJoin.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableJoin.java
new file mode 100644
index 0000000..4508c5a
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableJoin.java
@@ -0,0 +1,221 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.linq4j.function.Function2;
+import org.apache.calcite.linq4j.tree.BlockBuilder;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.linq4j.tree.ParameterExpression;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.InvalidRelException;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.JoinInfo;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rel.rules.EquiJoin;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.util.BuiltInMethod;
+import org.apache.calcite.util.ImmutableIntList;
+import org.apache.calcite.util.Util;
+
+import com.google.common.collect.ImmutableList;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+
+/** Implementation of {@link org.apache.calcite.rel.core.Join} in
+ * {@link org.apache.calcite.adapter.enumerable.EnumerableConvention enumerable calling convention}. */
+public class EnumerableJoin extends EquiJoin implements EnumerableRel {
+  EnumerableJoin(
+      RelOptCluster cluster,
+      RelTraitSet traits,
+      RelNode left,
+      RelNode right,
+      RexNode condition,
+      ImmutableIntList leftKeys,
+      ImmutableIntList rightKeys,
+      JoinRelType joinType,
+      Set<String> variablesStopped)
+      throws InvalidRelException {
+    super(
+        cluster,
+        traits,
+        left,
+        right,
+        condition,
+        leftKeys,
+        rightKeys,
+        joinType,
+        variablesStopped);
+  }
+
+  @Override public EnumerableJoin copy(RelTraitSet traitSet, RexNode condition,
+      RelNode left, RelNode right, JoinRelType joinType,
+      boolean semiJoinDone) {
+    final JoinInfo joinInfo = JoinInfo.of(left, right, condition);
+    assert joinInfo.isEqui();
+    try {
+      return new EnumerableJoin(getCluster(), traitSet, left, right,
+          condition, joinInfo.leftKeys, joinInfo.rightKeys, joinType,
+          variablesStopped);
+    } catch (InvalidRelException e) {
+      // Semantic error not possible. Must be a bug. Convert to
+      // internal error.
+      throw new AssertionError(e);
+    }
+  }
+
+  @Override public RelOptCost computeSelfCost(RelOptPlanner planner) {
+    double rowCount = RelMetadataQuery.getRowCount(this);
+
+    // Joins can be flipped, and for many algorithms, both versions are viable
+    // and have the same cost. To make the results stable between versions of
+    // the planner, make one of the versions slightly more expensive.
+    switch (joinType) {
+    case RIGHT:
+      rowCount = addEpsilon(rowCount);
+      break;
+    default:
+      if (left.getId() > right.getId()) {
+        rowCount = addEpsilon(rowCount);
+      }
+    }
+
+    // Cheaper if the smaller number of rows is coming from the LHS.
+    // Model this by adding L log L to the cost.
+    final double rightRowCount = right.getRows();
+    final double leftRowCount = left.getRows();
+    if (Double.isInfinite(leftRowCount)) {
+      rowCount = leftRowCount;
+    } else {
+      rowCount += Util.nLogN(leftRowCount);
+    }
+    if (Double.isInfinite(rightRowCount)) {
+      rowCount = rightRowCount;
+    } else {
+      rowCount += rightRowCount;
+    }
+    return planner.getCostFactory().makeCost(rowCount, 0, 0);
+  }
+
+  private double addEpsilon(double d) {
+    assert d >= 0d;
+    final double d0 = d;
+    if (d < 10) {
+      // For small d, adding 1 would change the value significantly.
+      d *= 1.001d;
+      if (d != d0) {
+        return d;
+      }
+    }
+    // For medium d, add 1. Keeps integral values integral.
+    ++d;
+    if (d != d0) {
+      return d;
+    }
+    // For large d, adding 1 might not change the value. Add .1%.
+    // If d is NaN, this still will probably not change the value. That's OK.
+    d *= 1.001d;
+    return d;
+  }
+
+  public Result implement(EnumerableRelImplementor implementor, Prefer pref) {
+    BlockBuilder builder = new BlockBuilder();
+    final Result leftResult =
+        implementor.visitChild(this, 0, (EnumerableRel) left, pref);
+    Expression leftExpression =
+        builder.append(
+            "left", leftResult.block);
+    final Result rightResult =
+        implementor.visitChild(this, 1, (EnumerableRel) right, pref);
+    Expression rightExpression =
+        builder.append(
+            "right", rightResult.block);
+    final PhysType physType =
+        PhysTypeImpl.of(
+            implementor.getTypeFactory(), getRowType(), pref.preferArray());
+    final PhysType keyPhysType =
+        leftResult.physType.project(
+            leftKeys, JavaRowFormat.LIST);
+    return implementor.result(
+        physType,
+        builder.append(
+            Expressions.call(
+                leftExpression,
+                BuiltInMethod.JOIN.method,
+                Expressions.list(
+                    rightExpression,
+                    leftResult.physType.generateAccessor(leftKeys),
+                    rightResult.physType.generateAccessor(rightKeys),
+                    generateSelector(
+                        physType,
+                        ImmutableList.of(
+                            leftResult.physType, rightResult.physType)))
+                    .append(
+                        Util.first(keyPhysType.comparer(),
+                            Expressions.constant(null)))
+                    .append(Expressions.constant(
+                        joinType.generatesNullsOnLeft()))
+                    .append(Expressions.constant(
+                        joinType.generatesNullsOnRight())))).toBlock());
+  }
+
+  Expression generateSelector(PhysType physType,
+      List<PhysType> inputPhysTypes) {
+    // A parameter for each input.
+    final List<ParameterExpression> parameters =
+        new ArrayList<ParameterExpression>();
+
+    // Generate all fields.
+    final List<Expression> expressions =
+        new ArrayList<Expression>();
+    for (Ord<PhysType> ord : Ord.zip(inputPhysTypes)) {
+      final PhysType inputPhysType =
+          ord.e.makeNullable(joinType.generatesNullsOn(ord.i));
+      final ParameterExpression parameter =
+          Expressions.parameter(inputPhysType.getJavaRowType(),
+              EnumUtils.LEFT_RIGHT[ord.i]);
+      parameters.add(parameter);
+      final int fieldCount = inputPhysType.getRowType().getFieldCount();
+      for (int i = 0; i < fieldCount; i++) {
+        Expression expression =
+            inputPhysType.fieldReference(parameter, i,
+                physType.getJavaFieldType(i));
+        if (joinType.generatesNullsOn(ord.i)) {
+          expression =
+              Expressions.condition(
+                  Expressions.equal(parameter, Expressions.constant(null)),
+                  Expressions.constant(null),
+                  expression);
+        }
+        expressions.add(expression);
+      }
+    }
+    return Expressions.lambda(
+        Function2.class,
+        physType.record(expressions),
+        parameters);
+  }
+}
+
+// End EnumerableJoin.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableJoinRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableJoinRule.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableJoinRule.java
new file mode 100644
index 0000000..915b635
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableJoinRule.java
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.rel.InvalidRelException;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.core.JoinInfo;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.logical.LogicalJoin;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/** Planner rule that converts a
+ * {@link org.apache.calcite.rel.logical.LogicalJoin} relational expression
+ * {@link org.apache.calcite.adapter.enumerable.EnumerableConvention enumerable calling convention}. */
+class EnumerableJoinRule extends ConverterRule {
+  EnumerableJoinRule() {
+    super(
+        LogicalJoin.class,
+        Convention.NONE,
+        EnumerableConvention.INSTANCE,
+        "EnumerableJoinRule");
+  }
+
+  @Override public RelNode convert(RelNode rel) {
+    LogicalJoin join = (LogicalJoin) rel;
+    List<RelNode> newInputs = new ArrayList<RelNode>();
+    for (RelNode input : join.getInputs()) {
+      if (!(input.getConvention() instanceof EnumerableConvention)) {
+        input =
+            convert(
+                input,
+                input.getTraitSet()
+                    .replace(EnumerableConvention.INSTANCE));
+      }
+      newInputs.add(input);
+    }
+    final RelNode left = newInputs.get(0);
+    final RelNode right = newInputs.get(1);
+    final JoinInfo info = JoinInfo.of(left, right, join.getCondition());
+    if (!info.isEqui() && join.getJoinType() != JoinRelType.INNER) {
+      // EnumerableJoinRel only supports equi-join. We can put a filter on top
+      // if it is an inner join.
+      return null;
+    }
+    final RelOptCluster cluster = join.getCluster();
+    RelNode newRel;
+    try {
+      newRel = new EnumerableJoin(
+          cluster,
+          join.getTraitSet().replace(EnumerableConvention.INSTANCE),
+          left,
+          right,
+          info.getEquiCondition(left, right, cluster.getRexBuilder()),
+          info.leftKeys,
+          info.rightKeys,
+          join.getJoinType(),
+          join.getVariablesStopped());
+    } catch (InvalidRelException e) {
+      EnumerableRules.LOGGER.fine(e.toString());
+      return null;
+    }
+    if (!info.isEqui()) {
+      newRel = new EnumerableFilter(cluster, newRel.getTraitSet(),
+          newRel, info.getRemaining(cluster.getRexBuilder()));
+    }
+    return newRel;
+  }
+}
+
+// End EnumerableJoinRule.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableLimit.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableLimit.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableLimit.java
new file mode 100644
index 0000000..bdbeaad
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableLimit.java
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.linq4j.tree.BlockBuilder;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelWriter;
+import org.apache.calcite.rel.SingleRel;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.util.BuiltInMethod;
+
+import java.util.List;
+
+/** Relational expression that applies a limit and/or offset to its input. */
+public class EnumerableLimit extends SingleRel implements EnumerableRel {
+  private final RexNode offset;
+  private final RexNode fetch;
+
+  public EnumerableLimit(
+      RelOptCluster cluster,
+      RelTraitSet traitSet,
+      RelNode child,
+      RexNode offset,
+      RexNode fetch) {
+    super(cluster, traitSet, child);
+    this.offset = offset;
+    this.fetch = fetch;
+    assert getConvention() instanceof EnumerableConvention;
+    assert getConvention() == child.getConvention();
+  }
+
+  @Override public EnumerableLimit copy(
+      RelTraitSet traitSet,
+      List<RelNode> newInputs) {
+    return new EnumerableLimit(
+        getCluster(),
+        traitSet,
+        sole(newInputs),
+        offset,
+        fetch);
+  }
+
+  @Override public RelWriter explainTerms(RelWriter pw) {
+    return super.explainTerms(pw)
+        .itemIf("offset", offset, offset != null)
+        .itemIf("fetch", fetch, fetch != null);
+  }
+
+  public Result implement(EnumerableRelImplementor implementor, Prefer pref) {
+    final BlockBuilder builder = new BlockBuilder();
+    final EnumerableRel child = (EnumerableRel) getInput();
+    final Result result = implementor.visitChild(this, 0, child, pref);
+    final PhysType physType =
+        PhysTypeImpl.of(
+            implementor.getTypeFactory(),
+            getRowType(),
+            result.format);
+
+    Expression v = builder.append("child", result.block);
+    if (offset != null) {
+      v = builder.append(
+          "offset",
+          Expressions.call(
+              v,
+              BuiltInMethod.SKIP.method,
+              Expressions.constant(RexLiteral.intValue(offset))));
+    }
+    if (fetch != null) {
+      v = builder.append(
+          "fetch",
+          Expressions.call(
+              v,
+              BuiltInMethod.TAKE.method,
+              Expressions.constant(RexLiteral.intValue(fetch))));
+    }
+
+    builder.add(
+        Expressions.return_(
+            null,
+            v));
+    return implementor.result(physType, builder.toBlock());
+  }
+}
+
+// End EnumerableLimit.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableLimitRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableLimitRule.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableLimitRule.java
new file mode 100644
index 0000000..4f8ab10
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableLimitRule.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Sort;
+
+/**
+ * Rule to convert an {@link org.apache.calcite.rel.core.Sort} that has
+ * {@code offset} or {@code fetch} set to an
+ * {@link EnumerableLimit}
+ * on top of a "pure" {@code Sort} that has no offset or fetch.
+ */
+class EnumerableLimitRule extends RelOptRule {
+  EnumerableLimitRule() {
+    super(
+        operand(Sort.class, any()),
+        "EnumerableLimitRule");
+  }
+
+  @Override public void onMatch(RelOptRuleCall call) {
+    final Sort sort = call.rel(0);
+    if (sort.offset == null && sort.fetch == null) {
+      return;
+    }
+    final RelTraitSet traitSet =
+        sort.getTraitSet().replace(EnumerableConvention.INSTANCE);
+    RelNode input = sort.getInput();
+    if (!sort.getCollation().getFieldCollations().isEmpty()) {
+      // Create a sort with the same sort key, but no offset or fetch.
+      input = sort.copy(
+          sort.getTraitSet(),
+          input,
+          sort.getCollation(),
+          null,
+          null);
+    }
+    RelNode x = convert(
+        input,
+        input.getTraitSet().replace(EnumerableConvention.INSTANCE));
+    call.transformTo(
+        new EnumerableLimit(
+            sort.getCluster(),
+            traitSet,
+            x,
+            sort.offset,
+            sort.fetch));
+  }
+}
+
+// End EnumerableLimitRule.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableMinus.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableMinus.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableMinus.java
new file mode 100644
index 0000000..c783cab
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableMinus.java
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.linq4j.tree.BlockBuilder;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Minus;
+import org.apache.calcite.util.BuiltInMethod;
+
+import java.util.List;
+
+/** Implementation of {@link org.apache.calcite.rel.core.Minus} in
+ * {@link org.apache.calcite.adapter.enumerable.EnumerableConvention enumerable calling convention}. */
+public class EnumerableMinus extends Minus implements EnumerableRel {
+  public EnumerableMinus(RelOptCluster cluster, RelTraitSet traitSet,
+      List<RelNode> inputs, boolean all) {
+    super(cluster, traitSet, inputs, all);
+    assert !all;
+  }
+
+  public EnumerableMinus copy(RelTraitSet traitSet, List<RelNode> inputs,
+      boolean all) {
+    return new EnumerableMinus(getCluster(), traitSet, inputs, all);
+  }
+
+  public Result implement(EnumerableRelImplementor implementor, Prefer pref) {
+    final BlockBuilder builder = new BlockBuilder();
+    Expression minusExp = null;
+    for (Ord<RelNode> ord : Ord.zip(inputs)) {
+      EnumerableRel input = (EnumerableRel) ord.e;
+      final Result result = implementor.visitChild(this, ord.i, input, pref);
+      Expression childExp =
+          builder.append(
+              "child" + ord.i,
+              result.block);
+
+      if (minusExp == null) {
+        minusExp = childExp;
+      } else {
+        minusExp =
+            Expressions.call(
+                minusExp,
+                BuiltInMethod.EXCEPT.method,
+                childExp);
+      }
+
+      // Once the first input has chosen its format, ask for the same for
+      // other inputs.
+      pref = pref.of(result.format);
+    }
+
+    builder.add(minusExp);
+    final PhysType physType =
+        PhysTypeImpl.of(
+            implementor.getTypeFactory(),
+            getRowType(),
+            pref.prefer(JavaRowFormat.CUSTOM));
+    return implementor.result(physType, builder.toBlock());
+  }
+}
+
+// End EnumerableMinus.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableMinusRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableMinusRule.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableMinusRule.java
new file mode 100644
index 0000000..4cbf263
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableMinusRule.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.logical.LogicalMinus;
+
+/**
+ * Rule to convert an {@link org.apache.calcite.rel.logical.LogicalMinus} to an
+ * {@link EnumerableMinus}.
+ */
+class EnumerableMinusRule extends ConverterRule {
+  EnumerableMinusRule() {
+    super(LogicalMinus.class, Convention.NONE, EnumerableConvention.INSTANCE,
+        "EnumerableMinusRule");
+  }
+
+  public RelNode convert(RelNode rel) {
+    final LogicalMinus minus = (LogicalMinus) rel;
+    if (minus.all) {
+      return null; // EXCEPT ALL not implemented
+    }
+    final EnumerableConvention out = EnumerableConvention.INSTANCE;
+    final RelTraitSet traitSet =
+        rel.getTraitSet().replace(
+            EnumerableConvention.INSTANCE);
+    return new EnumerableMinus(rel.getCluster(), traitSet,
+        convertList(minus.getInputs(), out), false);
+  }
+}
+
+// End EnumerableMinusRule.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableOneRowRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableOneRowRule.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableOneRowRule.java
new file mode 100644
index 0000000..3602d7f
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableOneRowRule.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.logical.LogicalOneRow;
+import org.apache.calcite.rex.RexBuilder;
+
+import java.math.BigDecimal;
+import java.util.Collections;
+
+/** Planner rule that converts a
+ * {@link org.apache.calcite.rel.logical.LogicalOneRow} relational expression
+ * {@link org.apache.calcite.adapter.enumerable.EnumerableConvention enumerable calling convention}. */
+public class EnumerableOneRowRule extends ConverterRule {
+  EnumerableOneRowRule() {
+    super(LogicalOneRow.class, Convention.NONE, EnumerableConvention.INSTANCE,
+        "EnumerableOneRowRule");
+  }
+
+  public RelNode convert(RelNode rel) {
+    LogicalOneRow oneRow = (LogicalOneRow) rel;
+    RexBuilder rexBuilder = rel.getCluster().getRexBuilder();
+    return new EnumerableValues(
+        oneRow.getCluster(),
+        oneRow.getRowType(),
+        Collections.singletonList(
+            Collections.singletonList(
+                rexBuilder.makeExactLiteral(BigDecimal.ZERO))),
+        oneRow.getTraitSet().replace(EnumerableConvention.INSTANCE));
+  }
+}
+
+// End EnumerableOneRowRule.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableProject.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableProject.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableProject.java
new file mode 100644
index 0000000..f145776
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableProject.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexNode;
+
+import java.util.List;
+
+/** Implementation of {@link org.apache.calcite.rel.core.Project} in
+ * {@link org.apache.calcite.adapter.enumerable.EnumerableConvention enumerable calling convention}. */
+public class EnumerableProject extends Project implements EnumerableRel {
+  public EnumerableProject(
+      RelOptCluster cluster,
+      RelTraitSet traitSet,
+      RelNode child,
+      List<? extends RexNode> exps,
+      RelDataType rowType,
+      int flags) {
+    super(cluster, traitSet, child, exps, rowType, flags);
+    assert getConvention() instanceof EnumerableConvention;
+  }
+
+  public EnumerableProject copy(RelTraitSet traitSet, RelNode input,
+      List<RexNode> exps, RelDataType rowType) {
+    return new EnumerableProject(getCluster(), traitSet, input,
+        exps, rowType, flags);
+  }
+
+  public Result implement(EnumerableRelImplementor implementor, Prefer pref) {
+    // EnumerableCalcRel is always better
+    throw new UnsupportedOperationException();
+  }
+}
+
+// End EnumerableProject.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableProjectRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableProjectRule.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableProjectRule.java
new file mode 100644
index 0000000..919856c
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableProjectRule.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.logical.LogicalProject;
+import org.apache.calcite.rex.RexMultisetUtil;
+import org.apache.calcite.rex.RexOver;
+
+/**
+ * Rule to convert a {@link org.apache.calcite.rel.logical.LogicalProject} to an
+ * {@link EnumerableProject}.
+ */
+class EnumerableProjectRule extends ConverterRule {
+  EnumerableProjectRule() {
+    super(LogicalProject.class, Convention.NONE, EnumerableConvention.INSTANCE,
+        "EnumerableProjectRule");
+  }
+
+  public RelNode convert(RelNode rel) {
+    final LogicalProject project = (LogicalProject) rel;
+
+    if (EnumUtils.B
+        && RexMultisetUtil.containsMultiset(project.getProjects(), true)
+        || RexOver.containsOver(project.getProjects(), null)) {
+      return null;
+    }
+
+    return new EnumerableProject(
+        rel.getCluster(),
+        rel.getTraitSet().replace(EnumerableConvention.INSTANCE),
+        convert(
+            project.getInput(),
+            project.getInput().getTraitSet()
+                .replace(EnumerableConvention.INSTANCE)),
+        project.getProjects(),
+        project.getRowType(),
+        Project.Flags.BOXED);
+  }
+}
+
+// End EnumerableProjectRule.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableProjectToCalcRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableProjectToCalcRule.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableProjectToCalcRule.java
new file mode 100644
index 0000000..988eb46
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableProjectToCalcRule.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexProgram;
+
+import com.google.common.collect.ImmutableList;
+
+/** Variant of {@link org.apache.calcite.rel.rules.ProjectToCalcRule} for
+ * {@link org.apache.calcite.adapter.enumerable.EnumerableConvention enumerable calling convention}. */
+public class EnumerableProjectToCalcRule extends RelOptRule {
+  EnumerableProjectToCalcRule() {
+    super(operand(EnumerableProject.class, any()));
+  }
+
+  public void onMatch(RelOptRuleCall call) {
+    final EnumerableProject project = call.rel(0);
+    final RelNode child = project.getInput();
+    final RelDataType rowType = project.getRowType();
+    final RexProgram program =
+        RexProgram.create(child.getRowType(),
+            project.getProjects(),
+            null,
+            project.getRowType(),
+            project.getCluster().getRexBuilder());
+    final EnumerableCalc calc =
+        new EnumerableCalc(
+            project.getCluster(),
+            project.getTraitSet(),
+            child,
+            rowType,
+            program,
+            ImmutableList.<RelCollation>of());
+    call.transformTo(calc);
+  }
+}
+
+// End EnumerableProjectToCalcRule.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableRel.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableRel.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableRel.java
index 5a69057..8c74f73 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableRel.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableRel.java
@@ -14,24 +14,23 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.rules.java;
-
-import net.hydromatic.linq4j.expressions.BlockStatement;
-
-import org.eigenbase.rel.ProjectRelBase;
-import org.eigenbase.rel.RelFactories;
-import org.eigenbase.rel.RelNode;
-import org.eigenbase.relopt.RelOptCluster;
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.rex.RexNode;
-import org.eigenbase.rex.RexUtil;
-import org.eigenbase.sql.validate.SqlValidatorUtil;
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.linq4j.tree.BlockStatement;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.RelFactories;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.sql.validate.SqlValidatorUtil;
 
 import java.util.List;
 
 /**
  * A relational expression of one of the
- * {@link net.hydromatic.optiq.rules.java.EnumerableConvention} calling
+ * {@link org.apache.calcite.adapter.enumerable.EnumerableConvention} calling
  * conventions.
  */
 public interface EnumerableRel
@@ -39,7 +38,7 @@ public interface EnumerableRel
   RelFactories.FilterFactory FILTER_FACTORY =
       new RelFactories.FilterFactory() {
         public RelNode createFilter(RelNode child, RexNode condition) {
-          return new JavaRules.EnumerableFilterRel(child.getCluster(),
+          return new EnumerableFilter(child.getCluster(),
               child.getTraitSet(), child, condition);
         }
       };
@@ -54,9 +53,9 @@ public interface EnumerableRel
                   fieldNames == null ? null
                       : SqlValidatorUtil.uniquify(fieldNames,
                           SqlValidatorUtil.F_SUGGESTER));
-          return new JavaRules.EnumerableProjectRel(cluster,
+          return new EnumerableProject(cluster,
               child.getTraitSet(), child, exprs, rowType,
-              ProjectRelBase.Flags.BOXED);
+              Project.Flags.BOXED);
         }
       };
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableRelImplementor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableRelImplementor.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableRelImplementor.java
index 135582e..86637dc 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableRelImplementor.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableRelImplementor.java
@@ -14,19 +14,33 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.rules.java;
+package org.apache.calcite.adapter.enumerable;
 
-import net.hydromatic.linq4j.Enumerable;
-import net.hydromatic.linq4j.Queryable;
-import net.hydromatic.linq4j.expressions.*;
-
-import net.hydromatic.optiq.BuiltinMethod;
-import net.hydromatic.optiq.DataContext;
-import net.hydromatic.optiq.jdbc.JavaTypeFactoryImpl;
-import net.hydromatic.optiq.runtime.*;
-
-import org.eigenbase.rel.RelNode;
-import org.eigenbase.rex.RexBuilder;
+import org.apache.calcite.DataContext;
+import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
+import org.apache.calcite.linq4j.Enumerable;
+import org.apache.calcite.linq4j.Queryable;
+import org.apache.calcite.linq4j.tree.BlockBuilder;
+import org.apache.calcite.linq4j.tree.BlockStatement;
+import org.apache.calcite.linq4j.tree.Blocks;
+import org.apache.calcite.linq4j.tree.ClassDeclaration;
+import org.apache.calcite.linq4j.tree.ConditionalStatement;
+import org.apache.calcite.linq4j.tree.ConstantExpression;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.linq4j.tree.MemberDeclaration;
+import org.apache.calcite.linq4j.tree.MethodCallExpression;
+import org.apache.calcite.linq4j.tree.NewArrayExpression;
+import org.apache.calcite.linq4j.tree.NewExpression;
+import org.apache.calcite.linq4j.tree.ParameterExpression;
+import org.apache.calcite.linq4j.tree.Primitive;
+import org.apache.calcite.linq4j.tree.Types;
+import org.apache.calcite.linq4j.tree.Visitor;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.runtime.Bindable;
+import org.apache.calcite.runtime.Utilities;
+import org.apache.calcite.util.BuiltInMethod;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterables;
@@ -34,10 +48,14 @@ import com.google.common.collect.Iterables;
 import java.io.Serializable;
 import java.lang.reflect.Modifier;
 import java.lang.reflect.Type;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
 
 /**
- * Subclass of {@link org.eigenbase.relopt.RelImplementor} for relational
+ * Subclass of {@link org.apache.calcite.plan.RelImplementor} for relational
  * operators of {@link EnumerableConvention} calling convention.
  */
 public class EnumerableRelImplementor extends JavaRelImplementor {
@@ -87,12 +105,12 @@ public class EnumerableRelImplementor extends JavaRelImplementor {
         Expressions.methodDecl(
             Modifier.PUBLIC,
             Enumerable.class,
-            BuiltinMethod.BINDABLE_BIND.method.getName(),
+            BuiltInMethod.BINDABLE_BIND.method.getName(),
             Expressions.list(root0_),
             block));
     memberDeclarations.add(Expressions.methodDecl(Modifier.PUBLIC,
         Type.class,
-        BuiltinMethod.TYPED_GET_ELEMENT_TYPE.method.getName(),
+        BuiltInMethod.TYPED_GET_ELEMENT_TYPE.method.getName(),
         Collections.<ParameterExpression>emptyList(),
         Blocks.toFunctionBlock(Expressions.return_(null,
             Expressions.constant(result.physType.getJavaRowType())))));
@@ -363,7 +381,7 @@ public class EnumerableRelImplementor extends JavaRelImplementor {
   public Expression stash(RelNode child, Class clazz) {
     final ParameterExpression x = register(child, clazz);
     final Expression e = Expressions.call(getRootExpression(),
-        BuiltinMethod.DATA_CONTEXT_GET.method, Expressions.constant(x.name));
+        BuiltInMethod.DATA_CONTEXT_GET.method, Expressions.constant(x.name));
     return Expressions.convert_(e, clazz);
   }
 
@@ -380,8 +398,7 @@ public class EnumerableRelImplementor extends JavaRelImplementor {
       this.types = types;
     }
 
-    @Override
-    public Expression visit(
+    @Override public Expression visit(
         NewExpression newExpression,
         List<Expression> arguments,
         List<MemberDeclaration> memberDeclarations) {
@@ -392,8 +409,7 @@ public class EnumerableRelImplementor extends JavaRelImplementor {
           memberDeclarations);
     }
 
-    @Override
-    public Expression visit(NewArrayExpression newArrayExpression,
+    @Override public Expression visit(NewArrayExpression newArrayExpression,
         int dimension, Expression bound, List<Expression> expressions) {
       Type type = newArrayExpression.type;
       for (;;) {


[44/58] [abbrv] [partial] incubator-calcite git commit: [CALCITE-306] Standardize code style for "import package.*; "

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcQueryProvider.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcQueryProvider.java b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcQueryProvider.java
index d136075..14c9282 100644
--- a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcQueryProvider.java
+++ b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcQueryProvider.java
@@ -14,9 +14,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl.jdbc;
+package org.apache.calcite.adapter.jdbc;
 
-import net.hydromatic.linq4j.*;
+import org.apache.calcite.linq4j.Enumerator;
+import org.apache.calcite.linq4j.QueryProvider;
+import org.apache.calcite.linq4j.QueryProviderImpl;
+import org.apache.calcite.linq4j.Queryable;
 
 /**
  * Implementation of {@link QueryProvider} that talks to JDBC databases.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcRel.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcRel.java b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcRel.java
index 4a4f361..ba91d1b 100644
--- a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcRel.java
+++ b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcRel.java
@@ -14,9 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl.jdbc;
+package org.apache.calcite.adapter.jdbc;
 
-import org.eigenbase.rel.RelNode;
+import org.apache.calcite.rel.RelNode;
 
 /**
  * Relational expression that uses JDBC calling convention.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcRules.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcRules.java b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcRules.java
index 87bd005..f793a43 100644
--- a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcRules.java
+++ b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcRules.java
@@ -14,34 +14,92 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl.jdbc;
-
-import net.hydromatic.linq4j.Ord;
-import net.hydromatic.linq4j.Queryable;
-import net.hydromatic.linq4j.expressions.*;
-
-import net.hydromatic.optiq.ModifiableTable;
-import net.hydromatic.optiq.prepare.Prepare;
-import net.hydromatic.optiq.util.BitSets;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.rel.convert.ConverterRule;
-import org.eigenbase.rel.metadata.RelMetadataQuery;
-import org.eigenbase.rel.rules.EquiJoinRel;
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.rex.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.fun.SqlStdOperatorTable;
-import org.eigenbase.sql.parser.SqlParserPos;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.sql.validate.SqlValidatorUtil;
-import org.eigenbase.trace.EigenbaseTrace;
-import org.eigenbase.util.*;
+package org.apache.calcite.adapter.jdbc;
+
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.linq4j.Queryable;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelTrait;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.prepare.Prepare;
+import org.apache.calcite.rel.InvalidRelException;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelWriter;
+import org.apache.calcite.rel.SingleRel;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.core.Aggregate;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.core.Intersect;
+import org.apache.calcite.rel.core.JoinInfo;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.core.Minus;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.core.TableModify;
+import org.apache.calcite.rel.core.Union;
+import org.apache.calcite.rel.core.Values;
+import org.apache.calcite.rel.logical.LogicalAggregate;
+import org.apache.calcite.rel.logical.LogicalCalc;
+import org.apache.calcite.rel.logical.LogicalFilter;
+import org.apache.calcite.rel.logical.LogicalIntersect;
+import org.apache.calcite.rel.logical.LogicalJoin;
+import org.apache.calcite.rel.logical.LogicalMinus;
+import org.apache.calcite.rel.logical.LogicalProject;
+import org.apache.calcite.rel.logical.LogicalTableModify;
+import org.apache.calcite.rel.logical.LogicalUnion;
+import org.apache.calcite.rel.logical.LogicalValues;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rel.rules.EquiJoin;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexLocalRef;
+import org.apache.calcite.rex.RexMultisetUtil;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.schema.ModifiableTable;
+import org.apache.calcite.sql.JoinConditionType;
+import org.apache.calcite.sql.JoinType;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlDialect;
+import org.apache.calcite.sql.SqlFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlJoin;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlNodeList;
+import org.apache.calcite.sql.SqlSelect;
+import org.apache.calcite.sql.SqlSetOperator;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.InferTypes;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.ReturnTypes;
+import org.apache.calcite.sql.validate.SqlValidatorUtil;
+import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.ImmutableIntList;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.trace.CalciteTrace;
 
 import com.google.common.collect.ImmutableList;
 
-import java.util.*;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
 import java.util.logging.Logger;
 
 /**
@@ -53,7 +111,7 @@ public class JdbcRules {
   private JdbcRules() {
   }
 
-  protected static final Logger LOGGER = EigenbaseTrace.getPlannerTracer();
+  protected static final Logger LOGGER = CalciteTrace.getPlannerTracer();
 
   private static final SqlParserPos POS = SqlParserPos.ZERO;
 
@@ -134,18 +192,16 @@ public class JdbcRules {
   /** Rule that converts a join to JDBC. */
   private static class JdbcJoinRule extends JdbcConverterRule {
     private JdbcJoinRule(JdbcConvention out) {
-      super(JoinRel.class, Convention.NONE, out, "JdbcJoinRule");
+      super(LogicalJoin.class, Convention.NONE, out, "JdbcJoinRule");
     }
 
-    @Override
-    public RelNode convert(RelNode rel) {
-      JoinRel join = (JoinRel) rel;
+    @Override public RelNode convert(RelNode rel) {
+      LogicalJoin join = (LogicalJoin) rel;
       List<RelNode> newInputs = new ArrayList<RelNode>();
       for (RelNode input : join.getInputs()) {
         if (!(input.getConvention() == getOutTrait())) {
           input =
-              convert(
-                  input,
+              convert(input,
                   input.getTraitSet().replace(out));
         }
         newInputs.add(input);
@@ -153,11 +209,11 @@ public class JdbcRules {
       final JoinInfo joinInfo =
           JoinInfo.of(newInputs.get(0), newInputs.get(1), join.getCondition());
       if (!joinInfo.isEqui()) {
-        // JdbcJoinRel only supports equi-join
+        // JdbcJoin only supports equi-join
         return null;
       }
       try {
-        return new JdbcJoinRel(
+        return new JdbcJoin(
             join.getCluster(),
             join.getTraitSet().replace(out),
             newInputs.get(0),
@@ -175,10 +231,8 @@ public class JdbcRules {
   }
 
   /** Join operator implemented in JDBC convention. */
-  public static class JdbcJoinRel
-      extends EquiJoinRel
-      implements JdbcRel {
-    protected JdbcJoinRel(
+  public static class JdbcJoin extends EquiJoin implements JdbcRel {
+    protected JdbcJoin(
         RelOptCluster cluster,
         RelTraitSet traits,
         RelNode left,
@@ -193,14 +247,13 @@ public class JdbcRules {
           joinType, variablesStopped);
     }
 
-    @Override
-    public JdbcJoinRel copy(RelTraitSet traitSet, RexNode condition,
+    @Override public JdbcJoin copy(RelTraitSet traitSet, RexNode condition,
         RelNode left, RelNode right, JoinRelType joinType,
         boolean semiJoinDone) {
       final JoinInfo joinInfo = JoinInfo.of(left, right, condition);
       assert joinInfo.isEqui();
       try {
-        return new JdbcJoinRel(getCluster(), traitSet, left, right,
+        return new JdbcJoin(getCluster(), traitSet, left, right,
             condition, joinInfo.leftKeys, joinInfo.rightKeys, joinType,
             variablesStopped);
       } catch (InvalidRelException e) {
@@ -210,16 +263,14 @@ public class JdbcRules {
       }
     }
 
-    @Override
-    public RelOptCost computeSelfCost(RelOptPlanner planner) {
+    @Override public RelOptCost computeSelfCost(RelOptPlanner planner) {
       // We always "build" the
       double rowCount = RelMetadataQuery.getRowCount(this);
 
       return planner.getCostFactory().makeCost(rowCount, 0, 0);
     }
 
-    @Override
-    public double getRows() {
+    @Override public double getRows() {
       final boolean leftKey = left.isKey(BitSets.of(leftKeys));
       final boolean rightKey = right.isKey(BitSets.of(rightKeys));
       final double leftRowCount = left.getRows();
@@ -285,17 +336,16 @@ public class JdbcRules {
   }
 
   /**
-   * Rule to convert a {@link CalcRel} to an
-   * {@link JdbcCalcRel}.
+   * Rule to convert a {@link org.apache.calcite.rel.logical.LogicalCalc} to an
+   * {@link org.apache.calcite.adapter.jdbc.JdbcRules.JdbcCalc}.
    */
-  private static class JdbcCalcRule
-      extends JdbcConverterRule {
+  private static class JdbcCalcRule extends JdbcConverterRule {
     private JdbcCalcRule(JdbcConvention out) {
-      super(CalcRel.class, Convention.NONE, out, "JdbcCalcRule");
+      super(LogicalCalc.class, Convention.NONE, out, "JdbcCalcRule");
     }
 
     public RelNode convert(RelNode rel) {
-      final CalcRel calc = (CalcRel) rel;
+      final LogicalCalc calc = (LogicalCalc) rel;
 
       // If there's a multiset, let FarragoMultisetSplitter work on it
       // first.
@@ -303,23 +353,24 @@ public class JdbcRules {
         return null;
       }
 
-      return new JdbcCalcRel(rel.getCluster(), rel.getTraitSet().replace(out),
-          convert(calc.getChild(), calc.getTraitSet().replace(out)),
-          calc.getProgram(), ProjectRelBase.Flags.BOXED);
+      return new JdbcCalc(rel.getCluster(), rel.getTraitSet().replace(out),
+          convert(calc.getInput(), calc.getTraitSet().replace(out)),
+          calc.getProgram(), Project.Flags.BOXED);
     }
   }
 
-  /** Calc operator implemented in JDBC convention. */
-  public static class JdbcCalcRel extends SingleRel implements JdbcRel {
+  /** Calc operator implemented in JDBC convention.
+   *
+   * @see org.apache.calcite.rel.core.Calc */
+  public static class JdbcCalc extends SingleRel implements JdbcRel {
     private final RexProgram program;
 
     /**
-     * Values defined in {@link org.eigenbase.rel.ProjectRelBase.Flags}.
+     * Values defined in {@link org.apache.calcite.rel.core.Project.Flags}.
      */
     protected final int flags;
 
-    public JdbcCalcRel(
-        RelOptCluster cluster,
+    public JdbcCalc(RelOptCluster cluster,
         RelTraitSet traitSet,
         RelNode child,
         RexProgram program,
@@ -336,25 +387,24 @@ public class JdbcRules {
     }
 
     public double getRows() {
-      return FilterRel.estimateFilteredRows(
-          getChild(), program);
+      return LogicalFilter.estimateFilteredRows(getInput(), program);
     }
 
     public RelOptCost computeSelfCost(RelOptPlanner planner) {
       double dRows = RelMetadataQuery.getRowCount(this);
-      double dCpu = RelMetadataQuery.getRowCount(getChild())
+      double dCpu = RelMetadataQuery.getRowCount(getInput())
           * program.getExprCount();
       double dIo = 0;
       return planner.getCostFactory().makeCost(dRows, dCpu, dIo);
     }
 
     public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
-      return new JdbcCalcRel(getCluster(), traitSet, sole(inputs), program,
+      return new JdbcCalc(getCluster(), traitSet, sole(inputs), program,
           flags);
     }
 
     public JdbcImplementor.Result implement(JdbcImplementor implementor) {
-      JdbcImplementor.Result x = implementor.visitChild(0, getChild());
+      JdbcImplementor.Result x = implementor.visitChild(0, getInput());
       final JdbcImplementor.Builder builder =
           program.getCondition() != null
               ? x.builder(this, JdbcImplementor.Clause.FROM,
@@ -377,36 +427,35 @@ public class JdbcRules {
   }
 
   /**
-   * Rule to convert a {@link ProjectRel} to an
-   * {@link JdbcProjectRel}.
+   * Rule to convert a {@link org.apache.calcite.rel.logical.LogicalProject} to
+   * an {@link org.apache.calcite.adapter.jdbc.JdbcRules.JdbcProject}.
    */
-  private static class JdbcProjectRule
-      extends ConverterRule {
+  private static class JdbcProjectRule extends JdbcConverterRule {
     private JdbcProjectRule(JdbcConvention out) {
-      super(ProjectRel.class, Convention.NONE, out, "JdbcProjectRule");
+      super(LogicalProject.class, Convention.NONE, out, "JdbcProjectRule");
     }
 
     public RelNode convert(RelNode rel) {
-      final ProjectRel project = (ProjectRel) rel;
+      final LogicalProject project = (LogicalProject) rel;
 
-      return new JdbcProjectRel(
+      return new JdbcProject(
           rel.getCluster(),
-          rel.getTraitSet().replace(getOutConvention()),
+          rel.getTraitSet().replace(out),
           convert(
-              project.getChild(),
-              project.getChild().getTraitSet().replace(getOutConvention())),
+              project.getInput(),
+              project.getInput().getTraitSet().replace(out)),
           project.getProjects(),
           project.getRowType(),
-          ProjectRelBase.Flags.BOXED);
+          Project.Flags.BOXED);
     }
   }
 
-  /** Implementation of {@link org.eigenbase.rel.ProjectRel} in
+  /** Implementation of {@link org.apache.calcite.rel.logical.LogicalProject} in
    * {@link JdbcConvention jdbc calling convention}. */
-  public static class JdbcProjectRel
-      extends ProjectRelBase
+  public static class JdbcProject
+      extends Project
       implements JdbcRel {
-    public JdbcProjectRel(
+    public JdbcProject(
         RelOptCluster cluster,
         RelTraitSet traitSet,
         RelNode child,
@@ -417,15 +466,15 @@ public class JdbcRules {
       assert getConvention() instanceof JdbcConvention;
     }
 
-    @Override public JdbcProjectRel copy(RelTraitSet traitSet, RelNode input,
+    @Override public JdbcProject copy(RelTraitSet traitSet, RelNode input,
         List<RexNode> exps, RelDataType rowType) {
-      return new JdbcProjectRel(getCluster(), traitSet, input, exps, rowType,
+      return new JdbcProject(getCluster(), traitSet, input, exps, rowType,
           flags);
     }
 
     public JdbcImplementor.Result implement(JdbcImplementor implementor) {
-      JdbcImplementor.Result x = implementor.visitChild(0, getChild());
-      if (isStar(exps, getChild().getRowType())) {
+      JdbcImplementor.Result x = implementor.visitChild(0, getInput());
+      if (isStar(exps, getInput().getRowType())) {
         return x;
       }
       final JdbcImplementor.Builder builder =
@@ -441,34 +490,30 @@ public class JdbcRules {
   }
 
   /**
-   * Rule to convert a {@link FilterRel} to an
-   * {@link JdbcFilterRel}.
+   * Rule to convert a {@link org.apache.calcite.rel.logical.LogicalFilter} to
+   * an {@link org.apache.calcite.adapter.jdbc.JdbcRules.JdbcFilter}.
    */
-  private static class JdbcFilterRule
-      extends ConverterRule {
+  private static class JdbcFilterRule extends JdbcConverterRule {
     private JdbcFilterRule(JdbcConvention out) {
-      super(FilterRel.class, Convention.NONE, out, "JdbcFilterRule");
+      super(LogicalFilter.class, Convention.NONE, out, "JdbcFilterRule");
     }
 
     public RelNode convert(RelNode rel) {
-      final FilterRel filter = (FilterRel) rel;
+      final LogicalFilter filter = (LogicalFilter) rel;
 
-      return new JdbcFilterRel(
+      return new JdbcFilter(
           rel.getCluster(),
-          rel.getTraitSet().replace(getOutConvention()),
-          convert(
-              filter.getChild(),
-              filter.getChild().getTraitSet().replace(getOutConvention())),
+          rel.getTraitSet().replace(out),
+          convert(filter.getInput(),
+              filter.getInput().getTraitSet().replace(out)),
           filter.getCondition());
     }
   }
 
-  /** Implementation of {@link org.eigenbase.rel.FilterRel} in
+  /** Implementation of {@link org.apache.calcite.rel.core.Filter} in
    * {@link JdbcConvention jdbc calling convention}. */
-  public static class JdbcFilterRel
-      extends FilterRelBase
-      implements JdbcRel {
-    public JdbcFilterRel(
+  public static class JdbcFilter extends Filter implements JdbcRel {
+    public JdbcFilter(
         RelOptCluster cluster,
         RelTraitSet traitSet,
         RelNode child,
@@ -477,13 +522,13 @@ public class JdbcRules {
       assert getConvention() instanceof JdbcConvention;
     }
 
-    public JdbcFilterRel copy(RelTraitSet traitSet, RelNode input,
+    public JdbcFilter copy(RelTraitSet traitSet, RelNode input,
         RexNode condition) {
-      return new JdbcFilterRel(getCluster(), traitSet, input, condition);
+      return new JdbcFilter(getCluster(), traitSet, input, condition);
     }
 
     public JdbcImplementor.Result implement(JdbcImplementor implementor) {
-      JdbcImplementor.Result x = implementor.visitChild(0, getChild());
+      JdbcImplementor.Result x = implementor.visitChild(0, getInput());
       final JdbcImplementor.Builder builder =
           x.builder(this, JdbcImplementor.Clause.WHERE);
       builder.setWhere(builder.context.toSql(null, condition));
@@ -492,21 +537,21 @@ public class JdbcRules {
   }
 
   /**
-   * Rule to convert an {@link org.eigenbase.rel.AggregateRel} to an
-   * {@link JdbcAggregateRel}.
+   * Rule to convert a {@link org.apache.calcite.rel.logical.LogicalAggregate}
+   * to a {@link org.apache.calcite.adapter.jdbc.JdbcRules.JdbcAggregate}.
    */
   private static class JdbcAggregateRule extends JdbcConverterRule {
     private JdbcAggregateRule(JdbcConvention out) {
-      super(AggregateRel.class, Convention.NONE, out, "JdbcAggregateRule");
+      super(LogicalAggregate.class, Convention.NONE, out, "JdbcAggregateRule");
     }
 
     public RelNode convert(RelNode rel) {
-      final AggregateRel agg = (AggregateRel) rel;
+      final LogicalAggregate agg = (LogicalAggregate) rel;
       final RelTraitSet traitSet =
           agg.getTraitSet().replace(out);
       try {
-        return new JdbcAggregateRel(rel.getCluster(), traitSet,
-            convert(agg.getChild(), traitSet), agg.getGroupSet(),
+        return new JdbcAggregate(rel.getCluster(), traitSet,
+            convert(agg.getInput(), traitSet), agg.getGroupSet(),
             agg.getAggCallList());
       } catch (InvalidRelException e) {
         LOGGER.fine(e.toString());
@@ -516,9 +561,8 @@ public class JdbcRules {
   }
 
   /** Aggregate operator implemented in JDBC convention. */
-  public static class JdbcAggregateRel extends AggregateRelBase
-      implements JdbcRel {
-    public JdbcAggregateRel(
+  public static class JdbcAggregate extends Aggregate implements JdbcRel {
+    public JdbcAggregate(
         RelOptCluster cluster,
         RelTraitSet traitSet,
         RelNode child,
@@ -529,10 +573,10 @@ public class JdbcRules {
       assert getConvention() instanceof JdbcConvention;
     }
 
-    @Override public JdbcAggregateRel copy(RelTraitSet traitSet, RelNode input,
+    @Override public JdbcAggregate copy(RelTraitSet traitSet, RelNode input,
         BitSet groupSet, List<AggregateCall> aggCalls) {
       try {
-        return new JdbcAggregateRel(getCluster(), traitSet, input, groupSet,
+        return new JdbcAggregate(getCluster(), traitSet, input, groupSet,
             aggCalls);
       } catch (InvalidRelException e) {
         // Semantic error not possible. Must be a bug. Convert to
@@ -543,7 +587,7 @@ public class JdbcRules {
 
     public JdbcImplementor.Result implement(JdbcImplementor implementor) {
       // "select a, b, sum(x) from ( ... ) group by a, b"
-      final JdbcImplementor.Result x = implementor.visitChild(0, getChild());
+      final JdbcImplementor.Result x = implementor.visitChild(0, getInput());
       final JdbcImplementor.Builder builder =
           x.builder(this, JdbcImplementor.Clause.GROUP_BY);
       List<SqlNode> groupByList = Expressions.list();
@@ -567,31 +611,31 @@ public class JdbcRules {
   }
 
   /**
-   * Rule to convert an {@link org.eigenbase.rel.SortRel} to an
-   * {@link JdbcSortRel}.
+   * Rule to convert a {@link org.apache.calcite.rel.core.Sort} to an
+   * {@link org.apache.calcite.adapter.jdbc.JdbcRules.JdbcSort}.
    */
   private static class JdbcSortRule extends JdbcConverterRule {
     private JdbcSortRule(JdbcConvention out) {
-      super(SortRel.class, Convention.NONE, out, "JdbcSortRule");
+      super(Sort.class, Convention.NONE, out, "JdbcSortRule");
     }
 
     public RelNode convert(RelNode rel) {
-      final SortRel sort = (SortRel) rel;
+      final Sort sort = (Sort) rel;
       if (sort.offset != null || sort.fetch != null) {
         // Cannot implement "OFFSET n FETCH n" currently.
         return null;
       }
       final RelTraitSet traitSet = sort.getTraitSet().replace(out);
-      return new JdbcSortRel(rel.getCluster(), traitSet,
-          convert(sort.getChild(), traitSet), sort.getCollation());
+      return new JdbcSort(rel.getCluster(), traitSet,
+          convert(sort.getInput(), traitSet), sort.getCollation());
     }
   }
 
   /** Sort operator implemented in JDBC convention. */
-  public static class JdbcSortRel
-      extends SortRel
+  public static class JdbcSort
+      extends Sort
       implements JdbcRel {
-    public JdbcSortRel(
+    public JdbcSort(
         RelOptCluster cluster,
         RelTraitSet traitSet,
         RelNode child,
@@ -601,14 +645,13 @@ public class JdbcRules {
       assert getConvention() == child.getConvention();
     }
 
-    @Override
-    public JdbcSortRel copy(RelTraitSet traitSet, RelNode newInput,
+    @Override public JdbcSort copy(RelTraitSet traitSet, RelNode newInput,
         RelCollation newCollation) {
-      return new JdbcSortRel(getCluster(), traitSet, newInput, newCollation);
+      return new JdbcSort(getCluster(), traitSet, newInput, newCollation);
     }
 
     public JdbcImplementor.Result implement(JdbcImplementor implementor) {
-      final JdbcImplementor.Result x = implementor.visitChild(0, getChild());
+      final JdbcImplementor.Result x = implementor.visitChild(0, getInput());
       final JdbcImplementor.Builder builder =
           x.builder(this, JdbcImplementor.Clause.ORDER_BY);
       List<SqlNode> orderByList = Expressions.list();
@@ -637,29 +680,26 @@ public class JdbcRules {
           OperandTypes.ANY, SqlFunctionCategory.SYSTEM);
 
   /**
-   * Rule to convert an {@link org.eigenbase.rel.UnionRel} to a
-   * {@link JdbcUnionRel}.
+   * Rule to convert an {@link org.apache.calcite.rel.logical.LogicalUnion} to a
+   * {@link org.apache.calcite.adapter.jdbc.JdbcRules.JdbcUnion}.
    */
-  private static class JdbcUnionRule
-      extends JdbcConverterRule {
+  private static class JdbcUnionRule extends JdbcConverterRule {
     private JdbcUnionRule(JdbcConvention out) {
-      super(UnionRel.class, Convention.NONE, out, "JdbcUnionRule");
+      super(LogicalUnion.class, Convention.NONE, out, "JdbcUnionRule");
     }
 
     public RelNode convert(RelNode rel) {
-      final UnionRel union = (UnionRel) rel;
+      final LogicalUnion union = (LogicalUnion) rel;
       final RelTraitSet traitSet =
           union.getTraitSet().replace(out);
-      return new JdbcUnionRel(rel.getCluster(), traitSet,
+      return new JdbcUnion(rel.getCluster(), traitSet,
           convertList(union.getInputs(), out), union.all);
     }
   }
 
   /** Union operator implemented in JDBC convention. */
-  public static class JdbcUnionRel
-      extends UnionRelBase
-      implements JdbcRel {
-    public JdbcUnionRel(
+  public static class JdbcUnion extends Union implements JdbcRel {
+    public JdbcUnion(
         RelOptCluster cluster,
         RelTraitSet traitSet,
         List<RelNode> inputs,
@@ -667,13 +707,12 @@ public class JdbcRules {
       super(cluster, traitSet, inputs, all);
     }
 
-    public JdbcUnionRel copy(
+    public JdbcUnion copy(
         RelTraitSet traitSet, List<RelNode> inputs, boolean all) {
-      return new JdbcUnionRel(getCluster(), traitSet, inputs, all);
+      return new JdbcUnion(getCluster(), traitSet, inputs, all);
     }
 
-    @Override
-    public RelOptCost computeSelfCost(RelOptPlanner planner) {
+    @Override public RelOptCost computeSelfCost(RelOptPlanner planner) {
       return super.computeSelfCost(planner).multiplyBy(.1);
     }
 
@@ -686,31 +725,31 @@ public class JdbcRules {
   }
 
   /**
-   * Rule to convert an {@link org.eigenbase.rel.IntersectRel} to an
-   * {@link JdbcIntersectRel}.
+   * Rule to convert a {@link org.apache.calcite.rel.logical.LogicalIntersect}
+   * to a {@link org.apache.calcite.adapter.jdbc.JdbcRules.JdbcIntersect}.
    */
   private static class JdbcIntersectRule extends JdbcConverterRule {
     private JdbcIntersectRule(JdbcConvention out) {
-      super(IntersectRel.class, Convention.NONE, out, "JdbcIntersectRule");
+      super(LogicalIntersect.class, Convention.NONE, out, "JdbcIntersectRule");
     }
 
     public RelNode convert(RelNode rel) {
-      final IntersectRel intersect = (IntersectRel) rel;
+      final LogicalIntersect intersect = (LogicalIntersect) rel;
       if (intersect.all) {
         return null; // INTERSECT ALL not implemented
       }
       final RelTraitSet traitSet =
           intersect.getTraitSet().replace(out);
-      return new JdbcIntersectRel(rel.getCluster(), traitSet,
-          convertList(intersect.getInputs(), out), intersect.all);
+      return new JdbcIntersect(rel.getCluster(), traitSet,
+          convertList(intersect.getInputs(), out), false);
     }
   }
 
   /** Intersect operator implemented in JDBC convention. */
-  public static class JdbcIntersectRel
-      extends IntersectRelBase
+  public static class JdbcIntersect
+      extends Intersect
       implements JdbcRel {
-    public JdbcIntersectRel(
+    public JdbcIntersect(
         RelOptCluster cluster,
         RelTraitSet traitSet,
         List<RelNode> inputs,
@@ -719,9 +758,9 @@ public class JdbcRules {
       assert !all;
     }
 
-    public JdbcIntersectRel copy(
+    public JdbcIntersect copy(
         RelTraitSet traitSet, List<RelNode> inputs, boolean all) {
-      return new JdbcIntersectRel(getCluster(), traitSet, inputs, all);
+      return new JdbcIntersect(getCluster(), traitSet, inputs, all);
     }
 
     public JdbcImplementor.Result implement(JdbcImplementor implementor) {
@@ -734,43 +773,37 @@ public class JdbcRules {
   }
 
   /**
-   * Rule to convert an {@link org.eigenbase.rel.MinusRel} to an
-   * {@link JdbcMinusRel}.
+   * Rule to convert a {@link org.apache.calcite.rel.logical.LogicalMinus} to a
+   * {@link org.apache.calcite.adapter.jdbc.JdbcRules.JdbcMinus}.
    */
-  private static class JdbcMinusRule
-      extends JdbcConverterRule {
+  private static class JdbcMinusRule extends JdbcConverterRule {
     private JdbcMinusRule(JdbcConvention out) {
-      super(MinusRel.class, Convention.NONE, out, "JdbcMinusRule");
+      super(LogicalMinus.class, Convention.NONE, out, "JdbcMinusRule");
     }
 
     public RelNode convert(RelNode rel) {
-      final MinusRel minus = (MinusRel) rel;
+      final LogicalMinus minus = (LogicalMinus) rel;
       if (minus.all) {
         return null; // EXCEPT ALL not implemented
       }
       final RelTraitSet traitSet =
           rel.getTraitSet().replace(out);
-      return new JdbcMinusRel(rel.getCluster(), traitSet,
-          convertList(minus.getInputs(), out), minus.all);
+      return new JdbcMinus(rel.getCluster(), traitSet,
+          convertList(minus.getInputs(), out), false);
     }
   }
 
   /** Minus operator implemented in JDBC convention. */
-  public static class JdbcMinusRel
-      extends MinusRelBase
-      implements JdbcRel {
-    public JdbcMinusRel(
-        RelOptCluster cluster,
-        RelTraitSet traitSet,
-        List<RelNode> inputs,
-        boolean all) {
+  public static class JdbcMinus extends Minus implements JdbcRel {
+    public JdbcMinus(RelOptCluster cluster, RelTraitSet traitSet,
+        List<RelNode> inputs, boolean all) {
       super(cluster, traitSet, inputs, all);
       assert !all;
     }
 
-    public JdbcMinusRel copy(
-        RelTraitSet traitSet, List<RelNode> inputs, boolean all) {
-      return new JdbcMinusRel(getCluster(), traitSet, inputs, all);
+    public JdbcMinus copy(RelTraitSet traitSet, List<RelNode> inputs,
+        boolean all) {
+      return new JdbcMinus(getCluster(), traitSet, inputs, all);
     }
 
     public JdbcImplementor.Result implement(JdbcImplementor implementor) {
@@ -786,16 +819,15 @@ public class JdbcRules {
   public static class JdbcTableModificationRule extends JdbcConverterRule {
     private JdbcTableModificationRule(JdbcConvention out) {
       super(
-          TableModificationRel.class,
+          LogicalTableModify.class,
           Convention.NONE,
           out,
           "JdbcTableModificationRule");
     }
 
-    @Override
-    public RelNode convert(RelNode rel) {
-      final TableModificationRel modify =
-          (TableModificationRel) rel;
+    @Override public RelNode convert(RelNode rel) {
+      final LogicalTableModify modify =
+          (LogicalTableModify) rel;
       final ModifiableTable modifiableTable =
           modify.getTable().unwrap(ModifiableTable.class);
       if (modifiableTable == null
@@ -804,11 +836,11 @@ public class JdbcRules {
       }
       final RelTraitSet traitSet =
           modify.getTraitSet().replace(out);
-      return new JdbcTableModificationRel(
+      return new JdbcTableModify(
           modify.getCluster(), traitSet,
           modify.getTable(),
           modify.getCatalogReader(),
-          convert(modify.getChild(), traitSet),
+          convert(modify.getInput(), traitSet),
           modify.getOperation(),
           modify.getUpdateColumnList(),
           modify.isFlattened());
@@ -816,13 +848,10 @@ public class JdbcRules {
   }
 
   /** Table-modification operator implemented in JDBC convention. */
-  public static class JdbcTableModificationRel
-      extends TableModificationRelBase
-      implements JdbcRel {
+  public static class JdbcTableModify extends TableModify implements JdbcRel {
     private final Expression expression;
 
-    public JdbcTableModificationRel(
-        RelOptCluster cluster,
+    public JdbcTableModify(RelOptCluster cluster,
         RelTraitSet traits,
         RelOptTable table,
         Prepare.CatalogReader catalogReader,
@@ -830,8 +859,7 @@ public class JdbcRules {
         Operation operation,
         List<String> updateColumnList,
         boolean flattened) {
-      super(
-          cluster, traits, table, catalogReader, child, operation,
+      super(cluster, traits, table, catalogReader, child, operation,
           updateColumnList, flattened);
       assert child.getConvention() instanceof JdbcConvention;
       assert getConvention() instanceof JdbcConvention;
@@ -846,9 +874,8 @@ public class JdbcRules {
       }
     }
 
-    @Override
-    public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
-      return new JdbcTableModificationRel(
+    @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+      return new JdbcTableModify(
           getCluster(), traitSet, getTable(), getCatalogReader(),
           sole(inputs), getOperation(), getUpdateColumnList(),
           isFlattened());
@@ -862,41 +889,26 @@ public class JdbcRules {
   /** Rule that converts a values operator to JDBC. */
   public static class JdbcValuesRule extends JdbcConverterRule {
     private JdbcValuesRule(JdbcConvention out) {
-      super(
-          ValuesRel.class,
-          Convention.NONE,
-          out,
-          "JdbcValuesRule");
+      super(LogicalValues.class, Convention.NONE, out, "JdbcValuesRule");
     }
 
-    @Override
-    public RelNode convert(RelNode rel) {
-      ValuesRel valuesRel = (ValuesRel) rel;
-      return new JdbcValuesRel(
-          valuesRel.getCluster(),
-          valuesRel.getRowType(),
-          valuesRel.getTuples(),
-          valuesRel.getTraitSet().replace(out));
+    @Override public RelNode convert(RelNode rel) {
+      LogicalValues values = (LogicalValues) rel;
+      return new JdbcValues(values.getCluster(), values.getRowType(),
+          values.getTuples(), values.getTraitSet().replace(out));
     }
   }
 
   /** Values operator implemented in JDBC convention. */
-  public static class JdbcValuesRel
-      extends ValuesRelBase
-      implements JdbcRel {
-    JdbcValuesRel(
-        RelOptCluster cluster,
-        RelDataType rowType,
-        List<List<RexLiteral>> tuples,
-        RelTraitSet traitSet) {
+  public static class JdbcValues extends Values implements JdbcRel {
+    JdbcValues(RelOptCluster cluster, RelDataType rowType,
+        List<List<RexLiteral>> tuples, RelTraitSet traitSet) {
       super(cluster, rowType, tuples, traitSet);
     }
 
-    @Override
-    public RelNode copy(
-        RelTraitSet traitSet, List<RelNode> inputs) {
+    @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
       assert inputs.isEmpty();
-      return new JdbcValuesRel(
+      return new JdbcValues(
           getCluster(), rowType, tuples, traitSet);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcSchema.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcSchema.java b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcSchema.java
index 655fd87..7db25ae 100644
--- a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcSchema.java
+++ b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcSchema.java
@@ -14,23 +14,38 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl.jdbc;
-
-import net.hydromatic.linq4j.expressions.Expression;
-
-import net.hydromatic.optiq.*;
-import net.hydromatic.optiq.Table;
-
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.SqlDialect;
-import org.eigenbase.sql.type.SqlTypeFactoryImpl;
-import org.eigenbase.sql.type.SqlTypeName;
-import org.eigenbase.util.Util;
-
-import com.google.common.collect.*;
-
-import java.sql.*;
-import java.util.*;
+package org.apache.calcite.adapter.jdbc;
+
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeImpl;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.rel.type.RelProtoDataType;
+import org.apache.calcite.schema.Function;
+import org.apache.calcite.schema.Schema;
+import org.apache.calcite.schema.SchemaFactory;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.schema.Schemas;
+import org.apache.calcite.schema.Table;
+import org.apache.calcite.sql.SqlDialect;
+import org.apache.calcite.sql.type.SqlTypeFactoryImpl;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.util.Util;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableMultimap;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Multimap;
+
+import java.sql.Connection;
+import java.sql.DatabaseMetaData;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
 import javax.sql.DataSource;
 
 /**
@@ -357,7 +372,7 @@ public class JdbcSchema implements Schema {
   }
 
   /** Schema factory that creates a
-   * {@link net.hydromatic.optiq.impl.jdbc.JdbcSchema}.
+   * {@link org.apache.calcite.adapter.jdbc.JdbcSchema}.
    * This allows you to create a jdbc schema inside a model.json file.
    *
    * <pre>{@code
@@ -368,7 +383,7 @@ public class JdbcSchema implements Schema {
    *     {
    *       name: 'FOODMART_CLONE',
    *       type: 'custom',
-   *       factory: 'net.hydromatic.optiq.impl.clone.JdbcSchema$Factory',
+   *       factory: 'org.apache.calcite.adapter.jdbc.JdbcSchema$Factory',
    *       operand: {
    *         jdbcDriver: 'com.mysql.jdbc.Driver',
    *         jdbcUrl: 'jdbc:mysql://localhost/foodmart',

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcTable.java b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcTable.java
index a3368ad..135568b 100644
--- a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcTable.java
+++ b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcTable.java
@@ -14,41 +14,50 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl.jdbc;
-
-import net.hydromatic.linq4j.*;
-import net.hydromatic.linq4j.expressions.*;
-import net.hydromatic.linq4j.function.*;
-
-import net.hydromatic.optiq.*;
-import net.hydromatic.optiq.impl.AbstractTableQueryable;
-import net.hydromatic.optiq.impl.java.AbstractQueryableTable;
-import net.hydromatic.optiq.impl.java.JavaTypeFactory;
-import net.hydromatic.optiq.jdbc.OptiqConnection;
-import net.hydromatic.optiq.runtime.ResultSetEnumerable;
-
-import org.eigenbase.rel.RelNode;
-import org.eigenbase.relopt.RelOptTable;
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.reltype.RelDataTypeFactory;
-import org.eigenbase.reltype.RelDataTypeField;
-import org.eigenbase.reltype.RelProtoDataType;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.parser.SqlParserPos;
-import org.eigenbase.sql.pretty.SqlPrettyWriter;
-import org.eigenbase.sql.util.SqlString;
-import org.eigenbase.util.Pair;
-import org.eigenbase.util.Util;
+package org.apache.calcite.adapter.jdbc;
+
+import org.apache.calcite.adapter.java.AbstractQueryableTable;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.jdbc.CalciteConnection;
+import org.apache.calcite.linq4j.Enumerable;
+import org.apache.calcite.linq4j.Enumerator;
+import org.apache.calcite.linq4j.QueryProvider;
+import org.apache.calcite.linq4j.Queryable;
+import org.apache.calcite.linq4j.function.Function1;
+import org.apache.calcite.linq4j.function.Functions;
+import org.apache.calcite.linq4j.tree.Primitive;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rel.type.RelProtoDataType;
+import org.apache.calcite.runtime.ResultSetEnumerable;
+import org.apache.calcite.schema.Schema;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.schema.TranslatableTable;
+import org.apache.calcite.schema.impl.AbstractTableQueryable;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlNodeList;
+import org.apache.calcite.sql.SqlSelect;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.pretty.SqlPrettyWriter;
+import org.apache.calcite.sql.util.SqlString;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
 
 import java.sql.SQLException;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
 
 /**
  * Queryable that gets its data from a table within a JDBC connection.
  *
  * <p>The idea is not to read the whole table, however. The idea is to use
  * this as a building block for a query, by applying Queryable operators
- * such as {@link net.hydromatic.linq4j.Queryable#where(net.hydromatic.linq4j.function.Predicate2)}.
+ * such as
+ * {@link org.apache.calcite.linq4j.Queryable#where(org.apache.calcite.linq4j.function.Predicate2)}.
  * The resulting queryable can then be converted to a SQL query, which can be
  * executed efficiently on the JDBC server.</p>
  */
@@ -89,7 +98,7 @@ class JdbcTable extends AbstractQueryableTable implements TranslatableTable {
       } catch (SQLException e) {
         throw new RuntimeException(
             "Exception while reading definition of table '" + jdbcTableName
-            + "'", e);
+                + "'", e);
       }
     }
     return protoRowType.apply(typeFactory);
@@ -161,7 +170,7 @@ class JdbcTable extends AbstractQueryableTable implements TranslatableTable {
 
     public Enumerator<T> enumerator() {
       final JavaTypeFactory typeFactory =
-          ((OptiqConnection) queryProvider).getTypeFactory();
+          ((CalciteConnection) queryProvider).getTypeFactory();
       final SqlString sql = generateSql();
       //noinspection unchecked
       final Enumerable<T> enumerable = (Enumerable<T>) ResultSetEnumerable.of(

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcTableScan.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcTableScan.java b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcTableScan.java
index 0de9142..96cabc3 100644
--- a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcTableScan.java
+++ b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcTableScan.java
@@ -14,11 +14,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl.jdbc;
+package org.apache.calcite.adapter.jdbc;
 
-import org.eigenbase.rel.RelNode;
-import org.eigenbase.rel.TableAccessRelBase;
-import org.eigenbase.relopt.*;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.TableScan;
 
 import java.util.Collections;
 import java.util.List;
@@ -26,7 +28,7 @@ import java.util.List;
 /**
  * Relational expression representing a scan of a table in a JDBC data source.
  */
-public class JdbcTableScan extends TableAccessRelBase implements JdbcRel {
+public class JdbcTableScan extends TableScan implements JdbcRel {
   final JdbcTable jdbcTable;
 
   protected JdbcTableScan(
@@ -39,8 +41,7 @@ public class JdbcTableScan extends TableAccessRelBase implements JdbcRel {
     assert jdbcTable != null;
   }
 
-  @Override
-  public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+  @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
     assert inputs.isEmpty();
     return new JdbcTableScan(
         getCluster(), table, jdbcTable, (JdbcConvention) getConvention());

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcToEnumerableConverter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcToEnumerableConverter.java b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcToEnumerableConverter.java
index 6cdcba1..3fbf778 100644
--- a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcToEnumerableConverter.java
+++ b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcToEnumerableConverter.java
@@ -14,24 +14,35 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl.jdbc;
+package org.apache.calcite.adapter.jdbc;
 
-import net.hydromatic.linq4j.expressions.*;
-
-import net.hydromatic.optiq.BuiltinMethod;
-import net.hydromatic.optiq.Schemas;
-import net.hydromatic.optiq.impl.java.JavaTypeFactory;
-import net.hydromatic.optiq.prepare.OptiqPrepareImpl;
-import net.hydromatic.optiq.rules.java.*;
-import net.hydromatic.optiq.runtime.Hook;
-import net.hydromatic.optiq.runtime.SqlFunctions;
-
-import org.eigenbase.rel.RelNode;
-import org.eigenbase.rel.convert.ConverterRelImpl;
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.sql.SqlDialect;
-import org.eigenbase.sql.type.SqlTypeName;
+import org.apache.calcite.adapter.enumerable.EnumerableRel;
+import org.apache.calcite.adapter.enumerable.EnumerableRelImplementor;
+import org.apache.calcite.adapter.enumerable.JavaRowFormat;
+import org.apache.calcite.adapter.enumerable.PhysType;
+import org.apache.calcite.adapter.enumerable.PhysTypeImpl;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.linq4j.tree.BlockBuilder;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.linq4j.tree.ParameterExpression;
+import org.apache.calcite.linq4j.tree.Primitive;
+import org.apache.calcite.linq4j.tree.UnaryExpression;
+import org.apache.calcite.plan.ConventionTraitDef;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.prepare.CalcitePrepareImpl;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterImpl;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.runtime.Hook;
+import org.apache.calcite.runtime.SqlFunctions;
+import org.apache.calcite.schema.Schemas;
+import org.apache.calcite.sql.SqlDialect;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.util.BuiltInMethod;
 
 import java.lang.reflect.Method;
 import java.lang.reflect.Modifier;
@@ -46,7 +57,7 @@ import java.util.TimeZone;
  * Relational expression representing a scan of a table in a JDBC data source.
  */
 public class JdbcToEnumerableConverter
-    extends ConverterRelImpl
+    extends ConverterImpl
     implements EnumerableRel {
   protected JdbcToEnumerableConverter(
       RelOptCluster cluster,
@@ -55,14 +66,12 @@ public class JdbcToEnumerableConverter
     super(cluster, ConventionTraitDef.INSTANCE, traits, input);
   }
 
-  @Override
-  public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+  @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
     return new JdbcToEnumerableConverter(
         getCluster(), traitSet, sole(inputs));
   }
 
-  @Override
-  public RelOptCost computeSelfCost(RelOptPlanner planner) {
+  @Override public RelOptCost computeSelfCost(RelOptPlanner planner) {
     return super.computeSelfCost(planner).multiplyBy(.1);
   }
 
@@ -70,7 +79,7 @@ public class JdbcToEnumerableConverter
     // Generate:
     //   ResultSetEnumerable.of(schema.getDataSource(), "select ...")
     final BlockBuilder builder0 = new BlockBuilder(false);
-    final JdbcRel child = (JdbcRel) getChild();
+    final JdbcRel child = (JdbcRel) getInput();
     final PhysType physType =
         PhysTypeImpl.of(
             implementor.getTypeFactory(), getRowType(),
@@ -78,7 +87,7 @@ public class JdbcToEnumerableConverter
     final JdbcConvention jdbcConvention =
         (JdbcConvention) child.getConvention();
     String sql = generateSql(jdbcConvention.dialect);
-    if (OptiqPrepareImpl.DEBUG) {
+    if (CalcitePrepareImpl.DEBUG) {
       System.out.println("[" + sql + "]");
     }
     Hook.QUERY_PLAN.run(sql);
@@ -143,11 +152,11 @@ public class JdbcToEnumerableConverter
         builder0.append(
             "enumerable",
             Expressions.call(
-                BuiltinMethod.RESULT_SET_ENUMERABLE_OF.method,
+                BuiltInMethod.RESULT_SET_ENUMERABLE_OF.method,
                 Expressions.call(
                     Schemas.unwrap(jdbcConvention.expression,
                         JdbcSchema.class),
-                    BuiltinMethod.JDBC_SCHEMA_DATA_SOURCE.method),
+                    BuiltInMethod.JDBC_SCHEMA_DATA_SOURCE.method),
                 sql_,
                 rowBuilderFactory_));
     builder0.add(
@@ -213,7 +222,7 @@ public class JdbcToEnumerableConverter
           Expressions.call(resultSet_, jdbcGetMethod(primitive),
               Expressions.constant(i + 1)),
           java.sql.Array.class);
-      source = Expressions.call(BuiltinMethod.JDBC_ARRAY_TO_LIST.method, x);
+      source = Expressions.call(BuiltInMethod.JDBC_ARRAY_TO_LIST.method, x);
       break;
     default:
       source = Expressions.call(
@@ -230,20 +239,20 @@ public class JdbcToEnumerableConverter
     switch (sqlTypeName) {
     case DATE:
       return (nullable
-          ? BuiltinMethod.DATE_TO_INT_OPTIONAL
-          : BuiltinMethod.DATE_TO_INT).method;
+          ? BuiltInMethod.DATE_TO_INT_OPTIONAL
+          : BuiltInMethod.DATE_TO_INT).method;
     case TIME:
       return (nullable
-          ? BuiltinMethod.TIME_TO_INT_OPTIONAL
-          : BuiltinMethod.TIME_TO_INT).method;
+          ? BuiltInMethod.TIME_TO_INT_OPTIONAL
+          : BuiltInMethod.TIME_TO_INT).method;
     case TIMESTAMP:
       return (nullable
           ? (offset
-          ? BuiltinMethod.TIMESTAMP_TO_LONG_OPTIONAL_OFFSET
-          : BuiltinMethod.TIMESTAMP_TO_LONG_OPTIONAL)
+          ? BuiltInMethod.TIMESTAMP_TO_LONG_OPTIONAL_OFFSET
+          : BuiltInMethod.TIMESTAMP_TO_LONG_OPTIONAL)
           : (offset
-              ? BuiltinMethod.TIMESTAMP_TO_LONG_OFFSET
-              : BuiltinMethod.TIMESTAMP_TO_LONG)).method;
+              ? BuiltInMethod.TIMESTAMP_TO_LONG_OFFSET
+              : BuiltInMethod.TIMESTAMP_TO_LONG)).method;
     default:
       throw new AssertionError(sqlTypeName + ":" + nullable);
     }
@@ -252,11 +261,11 @@ public class JdbcToEnumerableConverter
   private Method getMethod2(SqlTypeName sqlTypeName) {
     switch (sqlTypeName) {
     case DATE:
-      return BuiltinMethod.RESULT_SET_GET_DATE2.method;
+      return BuiltInMethod.RESULT_SET_GET_DATE2.method;
     case TIME:
-      return BuiltinMethod.RESULT_SET_GET_TIME2.method;
+      return BuiltInMethod.RESULT_SET_GET_TIME2.method;
     case TIMESTAMP:
-      return BuiltinMethod.RESULT_SET_GET_TIMESTAMP2.method;
+      return BuiltInMethod.RESULT_SET_GET_TIMESTAMP2.method;
     default:
       throw new AssertionError(sqlTypeName);
     }
@@ -274,7 +283,7 @@ public class JdbcToEnumerableConverter
         new JdbcImplementor(dialect,
             (JavaTypeFactory) getCluster().getTypeFactory());
     final JdbcImplementor.Result result =
-        jdbcImplementor.visitChild(0, getChild());
+        jdbcImplementor.visitChild(0, getInput());
     return result.asQuery().toSqlString(dialect).getSql();
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcToEnumerableConverterRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcToEnumerableConverterRule.java b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcToEnumerableConverterRule.java
index 26d1402..a212c3b 100644
--- a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcToEnumerableConverterRule.java
+++ b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcToEnumerableConverterRule.java
@@ -14,13 +14,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl.jdbc;
+package org.apache.calcite.adapter.jdbc;
 
-import net.hydromatic.optiq.rules.java.EnumerableConvention;
-
-import org.eigenbase.rel.RelNode;
-import org.eigenbase.rel.convert.ConverterRule;
-import org.eigenbase.relopt.RelTraitSet;
+import org.apache.calcite.adapter.enumerable.EnumerableConvention;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
 
 /**
  * Rule to convert a relational expression from

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcUtils.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcUtils.java b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcUtils.java
index ed0975f..fddb7d3 100644
--- a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcUtils.java
+++ b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcUtils.java
@@ -14,27 +14,37 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl.jdbc;
+package org.apache.calcite.adapter.jdbc;
 
-import net.hydromatic.linq4j.expressions.Primitive;
-import net.hydromatic.linq4j.function.*;
+import org.apache.calcite.linq4j.function.Function0;
+import org.apache.calcite.linq4j.function.Function1;
+import org.apache.calcite.linq4j.tree.Primitive;
+import org.apache.calcite.sql.SqlDialect;
+import org.apache.calcite.util.DateTimeUtil;
+import org.apache.calcite.util.ImmutableNullableList;
+import org.apache.calcite.util.IntList;
+import org.apache.calcite.util.Pair;
 
-import org.eigenbase.sql.SqlDialect;
-import org.eigenbase.util.ImmutableNullableList;
-import org.eigenbase.util.IntList;
-import org.eigenbase.util.Pair;
-import org.eigenbase.util14.DateTimeUtil;
+import org.apache.commons.dbcp.BasicDataSource;
 
 import com.google.common.cache.CacheBuilder;
 import com.google.common.cache.CacheLoader;
 import com.google.common.cache.LoadingCache;
 import com.google.common.collect.ImmutableList;
 
-import org.apache.commons.dbcp.BasicDataSource;
-
-import java.sql.*;
+import java.sql.Connection;
+import java.sql.DatabaseMetaData;
 import java.sql.Date;
-import java.util.*;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.sql.Types;
+import java.util.HashMap;
+import java.util.IdentityHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.TimeZone;
 import javax.sql.DataSource;
 
 /**
@@ -192,7 +202,7 @@ final class JdbcUtils {
    * the same object.
    *
    * <p>This in turn makes it easier to cache
-   * {@link org.eigenbase.sql.SqlDialect} objects. Otherwise, each time we
+   * {@link org.apache.calcite.sql.SqlDialect} objects. Otherwise, each time we
    * see a new data source, we have to open a connection to find out what
    * database product and version it is. */
   public static class DataSourcePool {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/jdbc/package-info.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/jdbc/package-info.java b/core/src/main/java/org/apache/calcite/adapter/jdbc/package-info.java
index 8ffc30d..92990dd 100644
--- a/core/src/main/java/org/apache/calcite/adapter/jdbc/package-info.java
+++ b/core/src/main/java/org/apache/calcite/adapter/jdbc/package-info.java
@@ -18,6 +18,6 @@
 /**
  * Query provider based on a JDBC data source.
  */
-package net.hydromatic.optiq.impl.jdbc;
+package org.apache.calcite.adapter.jdbc;
 
 // End package-info.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/package-info.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/package-info.java b/core/src/main/java/org/apache/calcite/adapter/package-info.java
index 052b587..697412a 100644
--- a/core/src/main/java/org/apache/calcite/adapter/package-info.java
+++ b/core/src/main/java/org/apache/calcite/adapter/package-info.java
@@ -16,8 +16,27 @@
  */
 
 /**
- * Utilities to help implement Calcite's SPIs.
+ * Calcite adapters.
+ *
+ * <p>An adapter allows Calcite to access data in a particular data source as
+ * if it were a collection of tables in a schema. Each adapter typically
+ * contains an implementation of {@link org.apache.calcite.schema.SchemaFactory}
+ * and some classes that implement other schema SPIs.
+ *
+ * <p>To use an adapter, include a custom schema in a JSON model file:
+ *
+ * <blockquote><pre>
+ *    schemas: [
+ *      {
+ *        type: 'custom',
+ *        name: 'My Custom Schema',
+ *        factory: 'com.acme.MySchemaFactory',
+ *        operand: {a: 'foo', b: [1, 3.5] }
+ *      }
+ *   ]
+ * </pre>
+ * </blockquote>
  */
-package net.hydromatic.optiq.impl;
+package org.apache.calcite.adapter;
 
 // End package-info.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/config/CalciteConnectionConfig.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/config/CalciteConnectionConfig.java b/core/src/main/java/org/apache/calcite/config/CalciteConnectionConfig.java
index 760b0d7..b47af97 100644
--- a/core/src/main/java/org/apache/calcite/config/CalciteConnectionConfig.java
+++ b/core/src/main/java/org/apache/calcite/config/CalciteConnectionConfig.java
@@ -14,38 +14,38 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.config;
+package org.apache.calcite.config;
 
-import net.hydromatic.avatica.Casing;
-import net.hydromatic.avatica.ConnectionConfig;
-import net.hydromatic.avatica.Quoting;
+import org.apache.calcite.avatica.Casing;
+import org.apache.calcite.avatica.ConnectionConfig;
+import org.apache.calcite.avatica.Quoting;
 
 /** Interface for reading connection properties within Calcite code. There is
  * a method for every property. At some point there will be similar config
  * classes for system and statement properties. */
-public interface OptiqConnectionConfig extends ConnectionConfig {
-  /** @see net.hydromatic.optiq.config.OptiqConnectionProperty#AUTO_TEMP */
+public interface CalciteConnectionConfig extends ConnectionConfig {
+  /** @see CalciteConnectionProperty#AUTO_TEMP */
   boolean autoTemp();
-  /** @see net.hydromatic.optiq.config.OptiqConnectionProperty#MATERIALIZATIONS_ENABLED */
+  /** @see CalciteConnectionProperty#MATERIALIZATIONS_ENABLED */
   boolean materializationsEnabled();
-  /** @see net.hydromatic.optiq.config.OptiqConnectionProperty#CREATE_MATERIALIZATIONS */
+  /** @see CalciteConnectionProperty#CREATE_MATERIALIZATIONS */
   boolean createMaterializations();
-  /** @see net.hydromatic.optiq.config.OptiqConnectionProperty#MODEL */
+  /** @see CalciteConnectionProperty#MODEL */
   String model();
-  /** @see net.hydromatic.optiq.config.OptiqConnectionProperty#LEX */
+  /** @see CalciteConnectionProperty#LEX */
   Lex lex();
-  /** @see net.hydromatic.optiq.config.OptiqConnectionProperty#QUOTING */
+  /** @see CalciteConnectionProperty#QUOTING */
   Quoting quoting();
-  /** @see net.hydromatic.optiq.config.OptiqConnectionProperty#UNQUOTED_CASING */
+  /** @see CalciteConnectionProperty#UNQUOTED_CASING */
   Casing unquotedCasing();
-  /** @see net.hydromatic.optiq.config.OptiqConnectionProperty#QUOTED_CASING */
+  /** @see CalciteConnectionProperty#QUOTED_CASING */
   Casing quotedCasing();
-  /** @see net.hydromatic.optiq.config.OptiqConnectionProperty#CASE_SENSITIVE */
+  /** @see CalciteConnectionProperty#CASE_SENSITIVE */
   boolean caseSensitive();
-  /** @see net.hydromatic.optiq.config.OptiqConnectionProperty#SPARK */
+  /** @see CalciteConnectionProperty#SPARK */
   boolean spark();
-  /** @see net.hydromatic.optiq.config.OptiqConnectionProperty#TYPE_SYSTEM */
+  /** @see CalciteConnectionProperty#TYPE_SYSTEM */
   <T> T typeSystem(Class<T> typeSystemClass, T defaultTypeSystem);
 }
 
-// End OptiqConnectionConfig.java
+// End CalciteConnectionConfig.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/config/CalciteConnectionConfigImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/config/CalciteConnectionConfigImpl.java b/core/src/main/java/org/apache/calcite/config/CalciteConnectionConfigImpl.java
index f08cc7c..442a54f 100644
--- a/core/src/main/java/org/apache/calcite/config/CalciteConnectionConfigImpl.java
+++ b/core/src/main/java/org/apache/calcite/config/CalciteConnectionConfigImpl.java
@@ -14,79 +14,79 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.config;
+package org.apache.calcite.config;
 
-import net.hydromatic.avatica.Casing;
-import net.hydromatic.avatica.ConnectionConfigImpl;
-import net.hydromatic.avatica.Quoting;
+import org.apache.calcite.avatica.Casing;
+import org.apache.calcite.avatica.ConnectionConfigImpl;
+import org.apache.calcite.avatica.Quoting;
 
 import java.util.Properties;
 
-/** Implementation of {@link OptiqConnectionConfig}. */
-public class OptiqConnectionConfigImpl extends ConnectionConfigImpl
-    implements OptiqConnectionConfig {
-  public OptiqConnectionConfigImpl(Properties properties) {
+/** Implementation of {@link CalciteConnectionConfig}. */
+public class CalciteConnectionConfigImpl extends ConnectionConfigImpl
+    implements CalciteConnectionConfig {
+  public CalciteConnectionConfigImpl(Properties properties) {
     super(properties);
   }
 
   /** Returns a copy of this configuration with one property changed. */
-  public OptiqConnectionConfigImpl set(OptiqConnectionProperty property,
+  public CalciteConnectionConfigImpl set(CalciteConnectionProperty property,
       String value) {
     final Properties properties1 = new Properties(properties);
     properties1.setProperty(property.camelName(), value);
-    return new OptiqConnectionConfigImpl(properties1);
+    return new CalciteConnectionConfigImpl(properties1);
   }
 
   public boolean autoTemp() {
-    return OptiqConnectionProperty.AUTO_TEMP.wrap(properties).getBoolean();
+    return CalciteConnectionProperty.AUTO_TEMP.wrap(properties).getBoolean();
   }
 
   public boolean materializationsEnabled() {
-    return OptiqConnectionProperty.MATERIALIZATIONS_ENABLED.wrap(properties)
+    return CalciteConnectionProperty.MATERIALIZATIONS_ENABLED.wrap(properties)
         .getBoolean();
   }
 
   public boolean createMaterializations() {
-    return OptiqConnectionProperty.CREATE_MATERIALIZATIONS.wrap(properties)
+    return CalciteConnectionProperty.CREATE_MATERIALIZATIONS.wrap(properties)
         .getBoolean();
   }
 
   public String model() {
-    return OptiqConnectionProperty.MODEL.wrap(properties).getString();
+    return CalciteConnectionProperty.MODEL.wrap(properties).getString();
   }
 
   public Lex lex() {
-    return OptiqConnectionProperty.LEX.wrap(properties).getEnum(Lex.class);
+    return CalciteConnectionProperty.LEX.wrap(properties).getEnum(Lex.class);
   }
 
   public Quoting quoting() {
-    return OptiqConnectionProperty.QUOTING.wrap(properties)
+    return CalciteConnectionProperty.QUOTING.wrap(properties)
         .getEnum(Quoting.class, lex().quoting);
   }
 
   public Casing unquotedCasing() {
-    return OptiqConnectionProperty.UNQUOTED_CASING.wrap(properties)
+    return CalciteConnectionProperty.UNQUOTED_CASING.wrap(properties)
         .getEnum(Casing.class, lex().unquotedCasing);
   }
 
   public Casing quotedCasing() {
-    return OptiqConnectionProperty.QUOTED_CASING.wrap(properties)
+    return CalciteConnectionProperty.QUOTED_CASING.wrap(properties)
         .getEnum(Casing.class, lex().quotedCasing);
   }
 
   public boolean caseSensitive() {
-    return OptiqConnectionProperty.CASE_SENSITIVE.wrap(properties)
+    return CalciteConnectionProperty.CASE_SENSITIVE.wrap(properties)
         .getBoolean(lex().caseSensitive);
   }
 
   public boolean spark() {
-    return OptiqConnectionProperty.SPARK.wrap(properties).getBoolean();
+    return CalciteConnectionProperty.SPARK.wrap(properties).getBoolean();
   }
 
   public <T> T typeSystem(Class<T> typeSystemClass, T defaultTypeSystem) {
-    return OptiqConnectionProperty.TYPE_SYSTEM.wrap(properties)
+    return CalciteConnectionProperty.TYPE_SYSTEM.wrap(properties)
         .getPlugin(typeSystemClass, defaultTypeSystem);
   }
 }
 
-// End OptiqConnectionConfigImpl.java
+// End CalciteConnectionConfigImpl.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/config/CalciteConnectionProperty.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/config/CalciteConnectionProperty.java b/core/src/main/java/org/apache/calcite/config/CalciteConnectionProperty.java
index f1ed97d..88b2621 100644
--- a/core/src/main/java/org/apache/calcite/config/CalciteConnectionProperty.java
+++ b/core/src/main/java/org/apache/calcite/config/CalciteConnectionProperty.java
@@ -14,20 +14,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.config;
+package org.apache.calcite.config;
 
-import net.hydromatic.avatica.ConnectionProperty;
+import org.apache.calcite.avatica.ConnectionProperty;
 
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Properties;
 
-import static net.hydromatic.avatica.ConnectionConfigImpl.*;
+import static org.apache.calcite.avatica.ConnectionConfigImpl.PropEnv;
+import static org.apache.calcite.avatica.ConnectionConfigImpl.parse;
 
 /**
  * Properties that may be specified on the JDBC connect string.
  */
-public enum OptiqConnectionProperty implements ConnectionProperty {
+public enum CalciteConnectionProperty implements ConnectionProperty {
   /** Whether to store query results in temporary tables. */
   AUTO_TEMP("autoTemp", Type.BOOLEAN, false),
 
@@ -71,25 +72,25 @@ public enum OptiqConnectionProperty implements ConnectionProperty {
   TIMEZONE("timezone", Type.STRING, null),
 
   /** Type system. The name of a class that implements
-   * {@code org.eigenbase.reltype.RelDataTypeSystem} and has a public default
-   * constructor or an {@code INSTANCE} constant. */
+   * {@link org.apache.calcite.rel.type.RelDataTypeSystem} and has a public
+   * default constructor or an {@code INSTANCE} constant. */
   TYPE_SYSTEM("typeSystem", Type.PLUGIN, null);
 
   private final String camelName;
   private final Type type;
   private final Object defaultValue;
 
-  private static final Map<String, OptiqConnectionProperty> NAME_TO_PROPS;
+  private static final Map<String, CalciteConnectionProperty> NAME_TO_PROPS;
 
   static {
-    NAME_TO_PROPS = new HashMap<String, OptiqConnectionProperty>();
-    for (OptiqConnectionProperty property : OptiqConnectionProperty.values()) {
-      NAME_TO_PROPS.put(property.camelName.toUpperCase(), property);
-      NAME_TO_PROPS.put(property.name(), property);
+    NAME_TO_PROPS = new HashMap<String, CalciteConnectionProperty>();
+    for (CalciteConnectionProperty p : CalciteConnectionProperty.values()) {
+      NAME_TO_PROPS.put(p.camelName.toUpperCase(), p);
+      NAME_TO_PROPS.put(p.name(), p);
     }
   }
 
-  OptiqConnectionProperty(String camelName, Type type, Object defaultValue) {
+  CalciteConnectionProperty(String camelName, Type type, Object defaultValue) {
     this.camelName = camelName;
     this.type = type;
     this.defaultValue = defaultValue;
@@ -114,4 +115,4 @@ public enum OptiqConnectionProperty implements ConnectionProperty {
   }
 }
 
-// End OptiqConnectionProperty.java
+// End CalciteConnectionProperty.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/config/Lex.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/config/Lex.java b/core/src/main/java/org/apache/calcite/config/Lex.java
index 3e510fc..51a6b63 100644
--- a/core/src/main/java/org/apache/calcite/config/Lex.java
+++ b/core/src/main/java/org/apache/calcite/config/Lex.java
@@ -14,10 +14,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.config;
+package org.apache.calcite.config;
 
-import net.hydromatic.avatica.Casing;
-import net.hydromatic.avatica.Quoting;
+import org.apache.calcite.avatica.Casing;
+import org.apache.calcite.avatica.Quoting;
 
 /** Named, built-in lexical policy. A lexical policy describes how
  * identifiers are quoted, whether they are converted to upper- or

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/config/package-info.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/config/package-info.java b/core/src/main/java/org/apache/calcite/config/package-info.java
index 008613b..b6d2ddb 100644
--- a/core/src/main/java/org/apache/calcite/config/package-info.java
+++ b/core/src/main/java/org/apache/calcite/config/package-info.java
@@ -18,6 +18,6 @@
 /**
  * Configuration.
  */
-package net.hydromatic.optiq.config;
+package org.apache.calcite.config;
 
 // End package-info.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/interpreter/Context.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/interpreter/Context.java b/core/src/main/java/org/apache/calcite/interpreter/Context.java
index 3d2b2f1..fc80319 100644
--- a/core/src/main/java/org/apache/calcite/interpreter/Context.java
+++ b/core/src/main/java/org/apache/calcite/interpreter/Context.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl.interpreter;
+package org.apache.calcite.interpreter;
 
 /**
  * Context for executing a scalar expression in an interpreter.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/interpreter/FilterNode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/interpreter/FilterNode.java b/core/src/main/java/org/apache/calcite/interpreter/FilterNode.java
index 003952c..d5461c4 100644
--- a/core/src/main/java/org/apache/calcite/interpreter/FilterNode.java
+++ b/core/src/main/java/org/apache/calcite/interpreter/FilterNode.java
@@ -14,12 +14,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl.interpreter;
+package org.apache.calcite.interpreter;
 
-import org.eigenbase.rel.FilterRelBase;
+import org.apache.calcite.rel.core.Filter;
 
 /**
- * Interpreter node that implements a {@link org.eigenbase.rel.FilterRelBase}.
+ * Interpreter node that implements a
+ * {@link org.apache.calcite.rel.core.Filter}.
  */
 public class FilterNode implements Node {
   private final Scalar condition;
@@ -27,7 +28,7 @@ public class FilterNode implements Node {
   private final Sink sink;
   private final Context context;
 
-  public FilterNode(Interpreter interpreter, FilterRelBase rel) {
+  public FilterNode(Interpreter interpreter, Filter rel) {
     this.condition = interpreter.compile(rel.getCondition());
     this.source = interpreter.source(rel, 0);
     this.sink = interpreter.sink(rel);

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/interpreter/Interpreter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/interpreter/Interpreter.java b/core/src/main/java/org/apache/calcite/interpreter/Interpreter.java
index 7607319..cbcfa64 100644
--- a/core/src/main/java/org/apache/calcite/interpreter/Interpreter.java
+++ b/core/src/main/java/org/apache/calcite/interpreter/Interpreter.java
@@ -14,26 +14,32 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl.interpreter;
-
-import net.hydromatic.linq4j.AbstractEnumerable;
-import net.hydromatic.linq4j.Enumerator;
-
-import net.hydromatic.optiq.DataContext;
-import net.hydromatic.optiq.prepare.OptiqPrepareImpl;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.rex.*;
-import org.eigenbase.util.ReflectUtil;
-import org.eigenbase.util.ReflectiveVisitDispatcher;
-import org.eigenbase.util.ReflectiveVisitor;
+package org.apache.calcite.interpreter;
+
+import org.apache.calcite.DataContext;
+import org.apache.calcite.linq4j.AbstractEnumerable;
+import org.apache.calcite.linq4j.Enumerator;
+import org.apache.calcite.prepare.CalcitePrepareImpl;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelVisitor;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.util.ReflectUtil;
+import org.apache.calcite.util.ReflectiveVisitDispatcher;
+import org.apache.calcite.util.ReflectiveVisitor;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 
 import java.math.BigDecimal;
-import java.util.*;
+import java.util.AbstractList;
+import java.util.ArrayDeque;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
 
 /**
  * Interpreter.
@@ -271,8 +277,8 @@ public class Interpreter extends AbstractEnumerable<Object[]> {
   }
 
   /**
-   * Walks over a tree of {@link org.eigenbase.rel.RelNode} and, for each,
-   * creates a {@link net.hydromatic.optiq.impl.interpreter.Node} that can be
+   * Walks over a tree of {@link org.apache.calcite.rel.RelNode} and, for each,
+   * creates a {@link org.apache.calcite.interpreter.Node} that can be
    * executed in the interpreter.
    *
    * <p>The compiler looks for methods of the form "visit(XxxRel)".
@@ -315,7 +321,7 @@ public class Interpreter extends AbstractEnumerable<Object[]> {
         if (rel == null) {
           break;
         }
-        if (OptiqPrepareImpl.DEBUG) {
+        if (CalcitePrepareImpl.DEBUG) {
           System.out.println("Interpreter: rewrite " + p + " to " + rel);
         }
         p = rel;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/interpreter/Node.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/interpreter/Node.java b/core/src/main/java/org/apache/calcite/interpreter/Node.java
index 8fd0a2d..cd24c5c 100644
--- a/core/src/main/java/org/apache/calcite/interpreter/Node.java
+++ b/core/src/main/java/org/apache/calcite/interpreter/Node.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl.interpreter;
+package org.apache.calcite.interpreter;
 
 /**
  * Relational expression that can be executed using an interpreter.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/interpreter/Nodes.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/interpreter/Nodes.java b/core/src/main/java/org/apache/calcite/interpreter/Nodes.java
index 7c5b43b..4f4cfd6 100644
--- a/core/src/main/java/org/apache/calcite/interpreter/Nodes.java
+++ b/core/src/main/java/org/apache/calcite/interpreter/Nodes.java
@@ -14,21 +14,26 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl.interpreter;
-
-import net.hydromatic.optiq.FilterableTable;
-import net.hydromatic.optiq.ProjectableFilterableTable;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.rel.rules.FilterTableRule;
-import org.eigenbase.relopt.RelOptCluster;
-import org.eigenbase.relopt.RelOptTable;
-import org.eigenbase.relopt.RelOptUtil;
-import org.eigenbase.relopt.RelTraitSet;
-import org.eigenbase.rex.RexNode;
-import org.eigenbase.util.ImmutableIntList;
-import org.eigenbase.util.Pair;
-import org.eigenbase.util.mapping.Mappings;
+package org.apache.calcite.interpreter;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Calc;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.rel.core.Values;
+import org.apache.calcite.rel.rules.FilterTableRule;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.schema.FilterableTable;
+import org.apache.calcite.schema.ProjectableFilterableTable;
+import org.apache.calcite.util.ImmutableIntList;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.mapping.Mappings;
 
 import com.google.common.collect.ImmutableList;
 
@@ -38,30 +43,30 @@ import com.google.common.collect.ImmutableList;
  */
 public class Nodes {
   /** Extension to
-   * {@link net.hydromatic.optiq.impl.interpreter.Interpreter.Compiler}
+   * {@link org.apache.calcite.interpreter.Interpreter.Compiler}
    * that knows how to handle the core logical
-   * {@link org.eigenbase.rel.RelNode}s. */
+   * {@link org.apache.calcite.rel.RelNode}s. */
   public static class CoreCompiler extends Interpreter.Compiler {
     CoreCompiler(Interpreter interpreter) {
       super(interpreter);
     }
 
-    public void rewrite(ProjectRelBase project) {
-      RelNode input = project.getChild();
+    public void rewrite(Project project) {
+      RelNode input = project.getInput();
       final Mappings.TargetMapping mapping = project.getMapping();
       if (mapping == null) {
         return;
       }
       RexNode condition;
-      if (input instanceof FilterRelBase) {
-        final FilterRelBase filter = (FilterRelBase) input;
+      if (input instanceof Filter) {
+        final Filter filter = (Filter) input;
         condition = filter.getCondition();
-        input = filter.getChild();
+        input = filter.getInput();
       } else {
         condition = project.getCluster().getRexBuilder().makeLiteral(true);
       }
-      if (input instanceof TableAccessRelBase) {
-        final TableAccessRelBase scan = (TableAccessRelBase) input;
+      if (input instanceof TableScan) {
+        final TableScan scan = (TableScan) input;
         final RelOptTable table = scan.getTable();
         final ProjectableFilterableTable projectableFilterableTable =
             table.unwrap(ProjectableFilterableTable.class);
@@ -69,7 +74,7 @@ public class Nodes {
           final FilterTableRule.FilterSplit filterSplit =
               FilterTableRule.FilterSplit.of(projectableFilterableTable,
                   condition, interpreter.getDataContext());
-          rel = new FilterScanRel(project.getCluster(), project.getTraitSet(),
+          rel = new FilterScan(project.getCluster(), project.getTraitSet(),
               table, filterSplit.acceptedFilters,
               ImmutableIntList.copyOf(Mappings.asList(mapping.inverse())));
           rel = RelOptUtil.createFilter(rel, filterSplit.rejectedFilters);
@@ -77,9 +82,9 @@ public class Nodes {
       }
     }
 
-    public void rewrite(FilterRelBase filter) {
-      if (filter.getChild() instanceof TableAccessRelBase) {
-        final TableAccessRelBase scan = (TableAccessRelBase) filter.getChild();
+    public void rewrite(Filter filter) {
+      if (filter.getInput() instanceof TableScan) {
+        final TableScan scan = (TableScan) filter.getInput();
         final RelOptTable table = scan.getTable();
         final ProjectableFilterableTable projectableFilterableTable =
             table.unwrap(ProjectableFilterableTable.class);
@@ -89,7 +94,7 @@ public class Nodes {
                   filter.getCondition(),
                   interpreter.getDataContext());
           if (!filterSplit.acceptedFilters.isEmpty()) {
-            rel = new FilterScanRel(scan.getCluster(), scan.getTraitSet(),
+            rel = new FilterScan(scan.getCluster(), scan.getTraitSet(),
                 table, filterSplit.acceptedFilters, null);
             rel = RelOptUtil.createFilter(rel, filterSplit.rejectedFilters);
             return;
@@ -103,7 +108,7 @@ public class Nodes {
                   filter.getCondition(),
                   interpreter.getDataContext());
           if (!filterSplit.acceptedFilters.isEmpty()) {
-            rel = new FilterScanRel(scan.getCluster(), scan.getTraitSet(),
+            rel = new FilterScan(scan.getCluster(), scan.getTraitSet(),
                 table, filterSplit.acceptedFilters, null);
             rel = RelOptUtil.createFilter(rel, filterSplit.rejectedFilters);
           }
@@ -111,52 +116,53 @@ public class Nodes {
       }
     }
 
-    public void visit(FilterRelBase filter) {
+    public void visit(Filter filter) {
       node = new FilterNode(interpreter, filter);
     }
 
-    public void visit(ProjectRelBase project) {
+    public void visit(Project project) {
       node = new ProjectNode(interpreter, project);
     }
 
     /** Per {@link #rewrite(RelNode)}, writes to {@link #rel}.
      *
-     * <p>We don't handle {@link CalcRelBase} directly. Expand to a
-     * {@link ProjectRelBase} on {@link FilterRelBase} (or just a
-     * {@link ProjectRelBase}). */
-    public void rewrite(CalcRelBase calc) {
+     * <p>We don't handle {@link org.apache.calcite.rel.core.Calc} directly.
+     * Expand to a {@link org.apache.calcite.rel.core.Project}
+     * on {@link org.apache.calcite.rel.core.Filter} (or just a
+     * {@link org.apache.calcite.rel.core.Project}). */
+    public void rewrite(Calc calc) {
       final Pair<ImmutableList<RexNode>, ImmutableList<RexNode>> projectFilter =
           calc.getProgram().split();
-      rel = calc.getChild();
+      rel = calc.getInput();
       rel = RelOptUtil.createFilter(rel, projectFilter.right);
       rel = RelOptUtil.createProject(rel, projectFilter.left,
           calc.getRowType().getFieldNames());
     }
 
-    public void visit(ValuesRelBase value) {
+    public void visit(Values value) {
       node = new ValuesNode(interpreter, value);
     }
 
-    public void visit(TableAccessRelBase scan) {
+    public void visit(TableScan scan) {
       node = new ScanNode(interpreter, scan, ImmutableList.<RexNode>of(), null);
     }
 
-    public void visit(FilterScanRel scan) {
+    public void visit(FilterScan scan) {
       node = new ScanNode(interpreter, scan, scan.filters, scan.projects);
     }
 
-    public void visit(SortRel sort) {
+    public void visit(Sort sort) {
       node = new SortNode(interpreter, sort);
     }
   }
 
   /** Table scan that applies filters and optionally projects. Only used in an
    * interpreter. */
-  public static class FilterScanRel extends TableAccessRelBase {
+  public static class FilterScan extends TableScan {
     private final ImmutableList<RexNode> filters;
     private final ImmutableIntList projects;
 
-    protected FilterScanRel(RelOptCluster cluster, RelTraitSet traits,
+    protected FilterScan(RelOptCluster cluster, RelTraitSet traits,
         RelOptTable table, ImmutableList<RexNode> filters,
         ImmutableIntList projects) {
       super(cluster, traits, table);

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/interpreter/ProjectNode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/interpreter/ProjectNode.java b/core/src/main/java/org/apache/calcite/interpreter/ProjectNode.java
index 1a2d597..bca464b 100644
--- a/core/src/main/java/org/apache/calcite/interpreter/ProjectNode.java
+++ b/core/src/main/java/org/apache/calcite/interpreter/ProjectNode.java
@@ -14,15 +14,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl.interpreter;
+package org.apache.calcite.interpreter;
 
-import org.eigenbase.rel.ProjectRelBase;
-import org.eigenbase.rex.RexNode;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rex.RexNode;
 
 import com.google.common.collect.ImmutableList;
 
 /**
- * Interpreter node that implements a {@link org.eigenbase.rel.FilterRel}.
+ * Interpreter node that implements a
+ * {@link org.apache.calcite.rel.logical.LogicalFilter}.
  */
 public class ProjectNode implements Node {
   private final ImmutableList<Scalar> projects;
@@ -30,7 +31,7 @@ public class ProjectNode implements Node {
   private final Sink sink;
   private final Context context;
 
-  public ProjectNode(Interpreter interpreter, ProjectRelBase rel) {
+  public ProjectNode(Interpreter interpreter, Project rel) {
     ImmutableList.Builder<Scalar> builder = ImmutableList.builder();
     for (RexNode node : rel.getProjects()) {
       builder.add(interpreter.compile(node));

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/interpreter/Row.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/interpreter/Row.java b/core/src/main/java/org/apache/calcite/interpreter/Row.java
index c84acc1..acead37 100644
--- a/core/src/main/java/org/apache/calcite/interpreter/Row.java
+++ b/core/src/main/java/org/apache/calcite/interpreter/Row.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl.interpreter;
+package org.apache.calcite.interpreter;
 
 import java.util.Arrays;
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/interpreter/Scalar.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/interpreter/Scalar.java b/core/src/main/java/org/apache/calcite/interpreter/Scalar.java
index b3e2118..2f7e923 100644
--- a/core/src/main/java/org/apache/calcite/interpreter/Scalar.java
+++ b/core/src/main/java/org/apache/calcite/interpreter/Scalar.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl.interpreter;
+package org.apache.calcite.interpreter;
 
 /**
  * Compiled scalar expression.


[41/58] [abbrv] [partial] incubator-calcite git commit: [CALCITE-306] Standardize code style for "import package.*; "

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/plan/RelOptLattice.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/RelOptLattice.java b/core/src/main/java/org/apache/calcite/plan/RelOptLattice.java
index be7049a..9131ba3 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptLattice.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptLattice.java
@@ -14,20 +14,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.relopt;
+package org.apache.calcite.plan;
+
+import org.apache.calcite.config.CalciteConnectionConfig;
+import org.apache.calcite.jdbc.CalciteSchema;
+import org.apache.calcite.materialize.Lattice;
+import org.apache.calcite.materialize.MaterializationService;
+import org.apache.calcite.materialize.TileKey;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.util.Pair;
 
 import java.util.BitSet;
 import java.util.List;
 
-import org.eigenbase.rel.RelNode;
-import org.eigenbase.util.Pair;
-
-import net.hydromatic.optiq.config.OptiqConnectionConfig;
-import net.hydromatic.optiq.jdbc.OptiqSchema;
-import net.hydromatic.optiq.materialize.Lattice;
-import net.hydromatic.optiq.materialize.MaterializationService;
-import net.hydromatic.optiq.materialize.TileKey;
-
 /**
  * Use of a lattice by the query optimizer.
  */
@@ -70,17 +69,17 @@ public class RelOptLattice {
    * @param measureList Calls to aggregate functions
    * @return Materialized table
    */
-  public Pair<OptiqSchema.TableEntry, TileKey> getAggregate(
+  public Pair<CalciteSchema.TableEntry, TileKey> getAggregate(
       RelOptPlanner planner, BitSet groupSet,
       List<Lattice.Measure> measureList) {
-    final OptiqConnectionConfig config =
-        planner.getContext().unwrap(OptiqConnectionConfig.class);
+    final CalciteConnectionConfig config =
+        planner.getContext().unwrap(CalciteConnectionConfig.class);
     if (config == null) {
       return null;
     }
     final MaterializationService service = MaterializationService.instance();
     boolean create = lattice.auto && config.createMaterializations();
-    final OptiqSchema schema = starRelOptTable.unwrap(OptiqSchema.class);
+    final CalciteSchema schema = starRelOptTable.unwrap(CalciteSchema.class);
     return service.defineTile(lattice, groupSet, measureList, schema, create,
         false);
   }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/plan/RelOptListener.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/RelOptListener.java b/core/src/main/java/org/apache/calcite/plan/RelOptListener.java
index c3cd375..42109ae 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptListener.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptListener.java
@@ -14,11 +14,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.relopt;
+package org.apache.calcite.plan;
 
-import java.util.*;
+import org.apache.calcite.rel.RelNode;
 
-import org.eigenbase.rel.*;
+import java.util.EventListener;
+import java.util.EventObject;
 
 /**
  * RelOptListener defines an interface for listening to events which occur

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/plan/RelOptMaterialization.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/RelOptMaterialization.java b/core/src/main/java/org/apache/calcite/plan/RelOptMaterialization.java
index 77883cc..f09e5fd 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptMaterialization.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptMaterialization.java
@@ -14,34 +14,39 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.relopt;
+package org.apache.calcite.plan;
 
-import java.util.List;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.rel.metadata.DefaultRelMetadataProvider;
-import org.eigenbase.rel.rules.AggregateFilterTransposeRule;
-import org.eigenbase.rel.rules.AggregateProjectMergeRule;
-import org.eigenbase.rel.rules.MergeProjectRule;
-import org.eigenbase.rel.rules.PullUpProjectsAboveJoinRule;
-import org.eigenbase.rel.rules.PushFilterPastJoinRule;
-import org.eigenbase.rel.rules.PushProjectPastFilterRule;
-import org.eigenbase.rex.RexNode;
-import org.eigenbase.rex.RexUtil;
-import org.eigenbase.sql.SqlExplainLevel;
-import org.eigenbase.util.Util;
-import org.eigenbase.util.mapping.Mappings;
-
-import net.hydromatic.optiq.Table;
-import net.hydromatic.optiq.impl.StarTable;
-import net.hydromatic.optiq.prepare.OptiqPrepareImpl;
-import net.hydromatic.optiq.tools.Program;
-import net.hydromatic.optiq.tools.Programs;
+import org.apache.calcite.prepare.CalcitePrepareImpl;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelShuttleImpl;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.rel.logical.LogicalJoin;
+import org.apache.calcite.rel.logical.LogicalTableScan;
+import org.apache.calcite.rel.metadata.DefaultRelMetadataProvider;
+import org.apache.calcite.rel.rules.AggregateFilterTransposeRule;
+import org.apache.calcite.rel.rules.AggregateProjectMergeRule;
+import org.apache.calcite.rel.rules.FilterJoinRule;
+import org.apache.calcite.rel.rules.JoinProjectTransposeRule;
+import org.apache.calcite.rel.rules.ProjectFilterTransposeRule;
+import org.apache.calcite.rel.rules.ProjectMergeRule;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.schema.Table;
+import org.apache.calcite.schema.impl.StarTable;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.tools.Program;
+import org.apache.calcite.tools.Programs;
+import org.apache.calcite.util.Util;
+import org.apache.calcite.util.mapping.Mappings;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 
+import java.util.List;
+
 /**
  * Records that a particular query is materialized by a particular table.
  */
@@ -71,9 +76,9 @@ public class RelOptMaterialization {
 
   /**
    * Converts a relational expression to one that uses a
-   * {@link net.hydromatic.optiq.impl.StarTable}.
+   * {@link org.apache.calcite.schema.impl.StarTable}.
    * The relational expression is already in leaf-join-form, per
-   * {@link #toLeafJoinForm(org.eigenbase.rel.RelNode)}.
+   * {@link #toLeafJoinForm(org.apache.calcite.rel.RelNode)}.
    */
   public static RelNode tryUseStar(RelNode rel,
       final RelOptTable starRelOptTable) {
@@ -81,8 +86,7 @@ public class RelOptMaterialization {
     assert starTable != null;
     RelNode rel2 = rel.accept(
         new RelShuttleImpl() {
-          @Override
-          public RelNode visit(TableAccessRelBase scan) {
+          @Override public RelNode visit(TableScan scan) {
             RelOptTable relOptTable = scan.getTable();
             final Table table = relOptTable.unwrap(Table.class);
             if (table.equals(starTable.tables.get(0))) {
@@ -100,14 +104,13 @@ public class RelOptMaterialization {
             return scan;
           }
 
-          @Override
-          public RelNode visit(JoinRel join) {
+          @Override public RelNode visit(LogicalJoin join) {
             for (;;) {
               RelNode rel = super.visit(join);
-              if (rel == join || !(rel instanceof JoinRel)) {
+              if (rel == join || !(rel instanceof LogicalJoin)) {
                 return rel;
               }
-              join = (JoinRel) rel;
+              join = (LogicalJoin) rel;
               final ProjectFilterTable left =
                   ProjectFilterTable.of(join.getLeft());
               if (left != null) {
@@ -124,9 +127,9 @@ public class RelOptMaterialization {
             }
           }
 
-          /** Throws a {@link org.eigenbase.util.Util.FoundOne} containing a
-           * {@link org.eigenbase.rel.TableAccessRel} on success.
-           * (Yes, an exception for normal operation.) */
+          /** Throws a {@link org.apache.calcite.util.Util.FoundOne} containing
+           * a {@link org.apache.calcite.rel.logical.LogicalTableScan} on
+           * success.  (Yes, an exception for normal operation.) */
           private void match(ProjectFilterTable left, ProjectFilterTable right,
               RelOptCluster cluster) {
             final Mappings.TargetMapping leftMapping = left.mapping();
@@ -146,7 +149,7 @@ public class RelOptMaterialization {
                           Mappings.offsetSource(rightMapping, offset),
                           leftMapping.getTargetCount()));
               final RelNode project = RelOptUtil.createProject(
-                  new TableAccessRel(cluster, leftRelOptTable),
+                  new LogicalTableScan(cluster, leftRelOptTable),
                   Mappings.asList(mapping.inverse()));
               final List<RexNode> conditions = Lists.newArrayList();
               if (left.condition != null) {
@@ -170,7 +173,7 @@ public class RelOptMaterialization {
                       Mappings.offsetSource(leftMapping, offset),
                       Mappings.offsetTarget(rightMapping, leftCount));
               final RelNode project = RelOptUtil.createProject(
-                  new TableAccessRel(cluster, rightRelOptTable),
+                  new LogicalTableScan(cluster, rightRelOptTable),
                   Mappings.asList(mapping.inverse()));
               final List<RexNode> conditions = Lists.newArrayList();
               if (left.condition != null) {
@@ -191,7 +194,7 @@ public class RelOptMaterialization {
       return rel;
     }
     final Program program = Programs.hep(
-        ImmutableList.of(PushProjectPastFilterRule.INSTANCE,
+        ImmutableList.of(ProjectFilterTransposeRule.INSTANCE,
             AggregateProjectMergeRule.INSTANCE,
             AggregateFilterTransposeRule.INSTANCE),
         false,
@@ -203,28 +206,28 @@ public class RelOptMaterialization {
   private static class ProjectFilterTable {
     final RexNode condition;
     final Mappings.TargetMapping mapping;
-    final TableAccessRelBase scan;
+    final TableScan scan;
 
     private ProjectFilterTable(RexNode condition,
-        Mappings.TargetMapping mapping, TableAccessRelBase scan) {
+        Mappings.TargetMapping mapping, TableScan scan) {
       this.condition = condition;
       this.mapping = mapping;
       this.scan = Preconditions.checkNotNull(scan);
     }
 
     static ProjectFilterTable of(RelNode node) {
-      if (node instanceof FilterRelBase) {
-        final FilterRelBase filter = (FilterRelBase) node;
-        return of2(filter.getCondition(), filter.getChild());
+      if (node instanceof Filter) {
+        final Filter filter = (Filter) node;
+        return of2(filter.getCondition(), filter.getInput());
       } else {
         return of2(null, node);
       }
     }
 
     private static ProjectFilterTable of2(RexNode condition, RelNode node) {
-      if (node instanceof ProjectRelBase) {
-        final ProjectRelBase project = (ProjectRelBase) node;
-        return of3(condition, project.getMapping(), project.getChild());
+      if (node instanceof Project) {
+        final Project project = (Project) node;
+        return of3(condition, project.getMapping(), project.getInput());
       } else {
         return of3(condition, null, node);
       }
@@ -232,9 +235,9 @@ public class RelOptMaterialization {
 
     private static ProjectFilterTable of3(RexNode condition,
         Mappings.TargetMapping mapping, RelNode node) {
-      if (node instanceof TableAccessRelBase) {
+      if (node instanceof TableScan) {
         return new ProjectFilterTable(condition, mapping,
-            (TableAccessRelBase) node);
+            (TableScan) node);
       } else {
         return null;
       }
@@ -253,25 +256,25 @@ public class RelOptMaterialization {
 
   /**
    * Converts a relational expression to a form where
-   * {@link org.eigenbase.rel.JoinRel}s are
+   * {@link org.apache.calcite.rel.logical.LogicalJoin}s are
    * as close to leaves as possible.
    */
   public static RelNode toLeafJoinForm(RelNode rel) {
     final Program program = Programs.hep(
         ImmutableList.of(
-            PullUpProjectsAboveJoinRule.RIGHT_PROJECT,
-            PullUpProjectsAboveJoinRule.LEFT_PROJECT,
-            PushFilterPastJoinRule.PushFilterIntoJoinRule.FILTER_ON_JOIN,
-            MergeProjectRule.INSTANCE),
+            JoinProjectTransposeRule.RIGHT_PROJECT,
+            JoinProjectTransposeRule.LEFT_PROJECT,
+            FilterJoinRule.FilterIntoJoinRule.FILTER_ON_JOIN,
+            ProjectMergeRule.INSTANCE),
         false,
         new DefaultRelMetadataProvider());
-    if (OptiqPrepareImpl.DEBUG) {
+    if (CalcitePrepareImpl.DEBUG) {
       System.out.println(
           RelOptUtil.dumpPlan(
               "before", rel, false, SqlExplainLevel.DIGEST_ATTRIBUTES));
     }
     final RelNode rel2 = program.run(null, rel, null);
-    if (OptiqPrepareImpl.DEBUG) {
+    if (CalcitePrepareImpl.DEBUG) {
       System.out.println(
           RelOptUtil.dumpPlan(
               "after", rel2, false, SqlExplainLevel.DIGEST_ATTRIBUTES));

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/plan/RelOptNode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/RelOptNode.java b/core/src/main/java/org/apache/calcite/plan/RelOptNode.java
index 61b7494..4117dd3 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptNode.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptNode.java
@@ -14,11 +14,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.relopt;
+package org.apache.calcite.plan;
 
-import java.util.List;
+import org.apache.calcite.rel.type.RelDataType;
 
-import org.eigenbase.reltype.RelDataType;
+import java.util.List;
 
 /**
  * Node in a planner.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/plan/RelOptPlanner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/RelOptPlanner.java b/core/src/main/java/org/apache/calcite/plan/RelOptPlanner.java
index 5f0048a..4ce8038 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptPlanner.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptPlanner.java
@@ -14,18 +14,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.relopt;
+package org.apache.calcite.plan;
 
-import java.util.List;
-import java.util.logging.*;
-import java.util.regex.*;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.metadata.CachingRelMetadataProvider;
+import org.apache.calcite.rel.metadata.RelMetadataProvider;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.util.CancelFlag;
+import org.apache.calcite.util.trace.CalciteTrace;
 
-import org.eigenbase.rel.*;
-import org.eigenbase.rel.metadata.*;
-import org.eigenbase.rex.RexBuilder;
-import org.eigenbase.rex.RexNode;
-import org.eigenbase.trace.*;
-import org.eigenbase.util.*;
+import java.util.List;
+import java.util.logging.Logger;
+import java.util.regex.Pattern;
 
 /**
  * A <code>RelOptPlanner</code> is a query optimizer: it transforms a relational
@@ -35,7 +37,7 @@ import org.eigenbase.util.*;
 public interface RelOptPlanner {
   //~ Static fields/initializers ---------------------------------------------
 
-  Logger LOGGER = EigenbaseTrace.getPlannerTracer();
+  Logger LOGGER = CalciteTrace.getPlannerTracer();
 
   //~ Methods ----------------------------------------------------------------
 
@@ -57,8 +59,8 @@ public interface RelOptPlanner {
    * Registers a rel trait definition. If the {@link RelTraitDef} has already
    * been registered, does nothing.
    *
-   * @return whether the RelTraitDef was added, as per {@link
-   * java.util.Collection#add}
+   * @return whether the RelTraitDef was added, as per
+   * {@link java.util.Collection#add}
    */
   boolean addRelTraitDef(RelTraitDef relTraitDef);
 
@@ -78,22 +80,25 @@ public interface RelOptPlanner {
   void clear();
 
   /**
-   * Registers a rule. If the rule has already been registered, does nothing.
-   * This method should determine if the given rule is a {@link
-   * org.eigenbase.rel.convert.ConverterRule} and pass the ConverterRule to
-   * all {@link #addRelTraitDef(RelTraitDef) registered} RelTraitDef
+   * Registers a rule.
+   *
+   * <p>If the rule has already been registered, does nothing.
+   * This method determines if the given rule is a
+   * {@link org.apache.calcite.rel.convert.ConverterRule} and pass the
+   * ConverterRule to all
+   * {@link #addRelTraitDef(RelTraitDef) registered} RelTraitDef
    * instances.
    *
-   * @return whether the rule was added, as per {@link
-   * java.util.Collection#add}
+   * @return whether the rule was added, as per
+   * {@link java.util.Collection#add}
    */
   boolean addRule(RelOptRule rule);
 
   /**
    * Removes a rule.
    *
-   * @return true if the rule was present, as per {@link
-   * java.util.Collection#remove(Object)}
+   * @return true if the rule was present, as per
+   * {@link java.util.Collection#remove(Object)}
    */
   boolean removeRule(RelOptRule rule);
 
@@ -146,7 +151,8 @@ public interface RelOptPlanner {
   /**
    * Defines a pair of relational expressions that are equivalent.
    *
-   * <p>Typically {@code tableRel} is a {@link TableAccessRel} representing a
+   * <p>Typically {@code tableRel} is a
+   * {@link org.apache.calcite.rel.logical.LogicalTableScan} representing a
    * table that is a materialized view and {@code queryRel} is the SQL
    * expression that populates that view. The intention is that
    * {@code tableRel} is cheaper to evaluate and therefore if the query being
@@ -177,7 +183,8 @@ public interface RelOptPlanner {
   RelNode findBestExp();
 
   /**
-   * Returns the factory that creates {@link org.eigenbase.relopt.RelOptCost}s.
+   * Returns the factory that creates
+   * {@link org.apache.calcite.plan.RelOptCost}s.
    */
   RelOptCostFactory getCostFactory();
 
@@ -312,7 +319,7 @@ public interface RelOptPlanner {
   }
 
   /**
-   * Thrown by {@link org.eigenbase.relopt.RelOptPlanner#findBestExp()}.
+   * Thrown by {@link org.apache.calcite.plan.RelOptPlanner#findBestExp()}.
    */
   class CannotPlanException extends RuntimeException {
     public CannotPlanException(String message) {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/plan/RelOptPredicateList.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/RelOptPredicateList.java b/core/src/main/java/org/apache/calcite/plan/RelOptPredicateList.java
index 96134dd..2cfdf13 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptPredicateList.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptPredicateList.java
@@ -14,9 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.relopt;
+package org.apache.calcite.plan;
 
-import org.eigenbase.rex.RexNode;
+import org.apache.calcite.rex.RexNode;
 
 import com.google.common.collect.ImmutableList;
 
@@ -71,4 +71,4 @@ public class RelOptPredicateList {
   }
 }
 
-// End RelOptPulledUpPredicates.java
+// End RelOptPredicateList.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/plan/RelOptQuery.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/RelOptQuery.java b/core/src/main/java/org/apache/calcite/plan/RelOptQuery.java
index d9404c3..c3bd49f 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptQuery.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptQuery.java
@@ -14,17 +14,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.relopt;
+package org.apache.calcite.plan;
 
-import java.util.*;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rex.RexBuilder;
 
-import org.eigenbase.rel.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.rex.*;
+import java.util.HashMap;
+import java.util.Map;
 
 /**
- * A <code>RelOptQuery</code> represents a set of {@link RelNode relational
- * expressions} which derive from the same <code>select</code> statement.
+ * A <code>RelOptQuery</code> represents a set of
+ * {@link RelNode relational expressions} which derive from the same
+ * <code>select</code> statement.
  */
 public class RelOptQuery {
   //~ Static fields/initializers ---------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/plan/RelOptRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/RelOptRule.java b/core/src/main/java/org/apache/calcite/plan/RelOptRule.java
index 1650950..193ca9b 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptRule.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptRule.java
@@ -14,11 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.relopt;
+package org.apache.calcite.plan;
 
-import java.util.*;
-
-import org.eigenbase.rel.*;
+import org.apache.calcite.rel.RelNode;
 
 import com.google.common.base.Function;
 import com.google.common.base.Predicate;
@@ -26,13 +24,16 @@ import com.google.common.base.Predicates;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 
+import java.util.ArrayList;
+import java.util.List;
+
 /**
  * A <code>RelOptRule</code> transforms an expression into another. It has a
  * list of {@link RelOptRuleOperand}s, which determine whether the rule can be
  * applied to a particular section of the tree.
  *
- * <p>The optimizer figures out which rules are applicable, then calls {@link
- * #onMatch} on each of them.</p>
+ * <p>The optimizer figures out which rules are applicable, then calls
+ * {@link #onMatch} on each of them.</p>
  */
 public abstract class RelOptRule {
   //~ Static fields/initializers ---------------------------------------------
@@ -182,11 +183,11 @@ public abstract class RelOptRule {
    * can have a variable number of children. For example, the rule to
    * eliminate empty children of a Union would have operands</p>
    *
-   * <blockquote>Operand(UnionRel, true, Operand(EmptyRel))</blockquote>
+   * <blockquote>Operand(Union, true, Operand(Empty))</blockquote>
    *
    * <p>and given the relational expressions</p>
    *
-   * <blockquote>UnionRel(FilterRel, EmptyRel, ProjectRel)</blockquote>
+   * <blockquote>Union(LogicalFilter, Empty, LogicalProject)</blockquote>
    *
    * <p>would fire the rule with arguments</p>
    *
@@ -347,9 +348,9 @@ public abstract class RelOptRule {
    * the rule, and before calling {@link #onMatch(RelOptRuleCall)}.
    *
    * <p>In implementations of {@link RelOptPlanner} which may queue up a
-   * matched {@link RelOptRuleCall} for a long time before calling {@link
-   * #onMatch(RelOptRuleCall)}, this method is beneficial because it allows
-   * the planner to discard rules earlier in the process.
+   * matched {@link RelOptRuleCall} for a long time before calling
+   * {@link #onMatch(RelOptRuleCall)}, this method is beneficial because it
+   * allows the planner to discard rules earlier in the process.
    *
    * <p>The default implementation of this method returns <code>true</code>.
    * It is acceptable for any implementation of this method to give a false
@@ -490,9 +491,8 @@ public abstract class RelOptRule {
       description = className.substring(punc + 1);
     }
     if (description.matches("[0-9]+")) {
-      throw new RuntimeException(
-          "Derived description of rule class " + className
-          + " is an integer, not valid. "
+      throw new RuntimeException("Derived description of rule class "
+          + className + " is an integer, not valid. "
           + "Supply a description manually.");
     }
     return description;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/plan/RelOptRuleCall.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/RelOptRuleCall.java b/core/src/main/java/org/apache/calcite/plan/RelOptRuleCall.java
index a81a522..7b36cb9 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptRuleCall.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptRuleCall.java
@@ -14,17 +14,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.relopt;
+package org.apache.calcite.plan;
 
-import java.util.*;
-import java.util.logging.*;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.trace.*;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.util.trace.CalciteTrace;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 
+import java.util.List;
+import java.util.Map;
+import java.util.logging.Logger;
+
 /**
  * A <code>RelOptRuleCall</code> is an invocation of a {@link RelOptRule} with a
  * set of {@link RelNode relational expression}s as arguments.
@@ -32,7 +33,7 @@ import com.google.common.collect.ImmutableMap;
 public abstract class RelOptRuleCall {
   //~ Static fields/initializers ---------------------------------------------
 
-  protected static final Logger LOGGER = EigenbaseTrace.getPlannerTracer();
+  protected static final Logger LOGGER = CalciteTrace.getPlannerTracer();
 
   /**
    * Generator for {@link #id} values.
@@ -43,7 +44,7 @@ public abstract class RelOptRuleCall {
 
   public final int id;
   private final RelOptRuleOperand operand0;
-  private final Map<RelNode, List<RelNode>> nodeChildren;
+  private final Map<RelNode, List<RelNode>> nodeInputs;
   public final RelOptRule rule;
   public final RelNode[] rels;
   private final RelOptPlanner planner;
@@ -58,9 +59,9 @@ public abstract class RelOptRuleCall {
    * @param operand      Root operand
    * @param rels         Array of relational expressions which matched each
    *                     operand
-   * @param nodeChildren For each node which matched with <code>
-   *                     matchAnyChildren</code>=true, a list of the node's
-   *                     children
+   * @param nodeInputs   For each node which matched with
+   *                     {@code matchAnyChildren}=true, a list of the node's
+   *                     inputs
    * @param parents      list of parent RelNodes corresponding to the first
    *                     relational expression in the array argument, if known;
    *                     otherwise, null
@@ -69,12 +70,12 @@ public abstract class RelOptRuleCall {
       RelOptPlanner planner,
       RelOptRuleOperand operand,
       RelNode[] rels,
-      Map<RelNode, List<RelNode>> nodeChildren,
+      Map<RelNode, List<RelNode>> nodeInputs,
       List<RelNode> parents) {
     this.id = nextId++;
     this.planner = planner;
     this.operand0 = operand;
-    this.nodeChildren = nodeChildren;
+    this.nodeInputs = nodeInputs;
     this.rule = operand.getRule();
     this.rels = rels;
     this.parents = parents;
@@ -85,8 +86,8 @@ public abstract class RelOptRuleCall {
       RelOptPlanner planner,
       RelOptRuleOperand operand,
       RelNode[] rels,
-      Map<RelNode, List<RelNode>> nodeChildren) {
-    this(planner, operand, rels, nodeChildren, null);
+      Map<RelNode, List<RelNode>> nodeInputs) {
+    this(planner, operand, rels, nodeInputs, null);
   }
 
   //~ Methods ----------------------------------------------------------------
@@ -147,13 +148,13 @@ public abstract class RelOptRuleCall {
    * rule.
    *
    * <p>If the policy of the operand which caused the match is not
-   * {@link org.eigenbase.relopt.RelOptRuleOperandChildPolicy#ANY},
+   * {@link org.apache.calcite.plan.RelOptRuleOperandChildPolicy#ANY},
    * the children will have their
    * own operands and therefore be easily available in the array returned by
    * the {@link #getRels} method, so this method returns null.
    *
    * <p>This method is for
-   * {@link org.eigenbase.relopt.RelOptRuleOperandChildPolicy#ANY},
+   * {@link org.apache.calcite.plan.RelOptRuleOperandChildPolicy#ANY},
    * which is generally used when a node can have a variable number of
    * children, and hence where the matched children are not retrievable by any
    * other means.
@@ -162,7 +163,7 @@ public abstract class RelOptRuleCall {
    * @return Children of relational expression
    */
   public List<RelNode> getChildRels(RelNode rel) {
-    return nodeChildren.get(rel);
+    return nodeInputs.get(rel);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/plan/RelOptRuleOperand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/RelOptRuleOperand.java b/core/src/main/java/org/apache/calcite/plan/RelOptRuleOperand.java
index c15ecc3..00c4c83 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptRuleOperand.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptRuleOperand.java
@@ -14,27 +14,27 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.relopt;
+package org.apache.calcite.plan;
 
-import java.util.*;
-
-import org.eigenbase.rel.RelNode;
-import org.eigenbase.util.*;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.util.Util;
 
 import com.google.common.base.Preconditions;
 import com.google.common.base.Predicate;
 import com.google.common.base.Predicates;
 import com.google.common.collect.ImmutableList;
 
+import java.util.List;
+
 /**
- * A <code>RelOptRuleOperand</code> determines whether a {@link
- * org.eigenbase.relopt.RelOptRule} can be applied to a particular expression.
+ * Operand that determines whether a {@link RelOptRule}
+ * can be applied to a particular expression.
  *
  * <p>For example, the rule to pull a filter up from the left side of a join
- * takes operands: <code>(Join (Filter) (Any))</code>.</p>
+ * takes operands: <code>Join(Filter, Any)</code>.</p>
  *
  * <p>Note that <code>children</code> means different things if it is empty or
- * it is <code>null</code>: <code>(Join (Filter <b>()</b>) (Any))</code> means
+ * it is <code>null</code>: <code>Join(Filter <b>()</b>, Any)</code> means
  * that, to match the rule, <code>Filter</code> must have no operands.</p>
  */
 public class RelOptRuleOperand {
@@ -69,7 +69,7 @@ public class RelOptRuleOperand {
    * {@link RelOptRule#none()},
    * {@link RelOptRule#any},
    * {@link RelOptRule#unordered},
-   * See {@link org.eigenbase.relopt.RelOptRuleOperandChildren} for more
+   * See {@link org.apache.calcite.plan.RelOptRuleOperandChildren} for more
    * details.</p>
    *
    * @param clazz    Class of relational expression to match (must not be null)
@@ -78,7 +78,8 @@ public class RelOptRuleOperand {
    *
    * @deprecated Use
    * {@link #RelOptRuleOperand(Class, RelTrait, com.google.common.base.Predicate, RelOptRuleOperandChildren)};
-   * will be removed after {@link Bug#upgrade(String) 0.9.2}
+   * will be removed after
+   * {@link org.apache.calcite.util.Bug#upgrade(String) 0.9.2}
    */
   protected <R extends RelNode> RelOptRuleOperand(
       Class<? extends R> clazz,
@@ -96,7 +97,7 @@ public class RelOptRuleOperand {
    * {@link RelOptRule#none()},
    * {@link RelOptRule#any},
    * {@link RelOptRule#unordered},
-   * See {@link org.eigenbase.relopt.RelOptRuleOperandChildren} for more
+   * See {@link org.apache.calcite.plan.RelOptRuleOperandChildren} for more
    * details.</p>
    *
    * @param clazz    Class of relational expression to match (must not be null)
@@ -189,7 +190,7 @@ public class RelOptRuleOperand {
     RelOptRuleOperand that = (RelOptRuleOperand) obj;
 
     return (this.clazz == that.clazz)
-        && Util.equal(this.trait, that.trait)
+        && com.google.common.base.Objects.equal(this.trait, that.trait)
         && this.children.equals(that.children);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/plan/RelOptRuleOperandChildPolicy.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/RelOptRuleOperandChildPolicy.java b/core/src/main/java/org/apache/calcite/plan/RelOptRuleOperandChildPolicy.java
index 594cd2b..a9bbc68 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptRuleOperandChildPolicy.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptRuleOperandChildPolicy.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.relopt;
+package org.apache.calcite.plan;
 
 /**
  * Policy by which operands will be matched by relational expressions with

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/plan/RelOptRuleOperandChildren.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/RelOptRuleOperandChildren.java b/core/src/main/java/org/apache/calcite/plan/RelOptRuleOperandChildren.java
index 1322da9..da007c6 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptRuleOperandChildren.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptRuleOperandChildren.java
@@ -14,13 +14,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.relopt;
+package org.apache.calcite.plan;
 
 import com.google.common.collect.ImmutableList;
 
 /**
- * Children of a {@link org.eigenbase.relopt.RelOptRuleOperand} and the policy
- * for matching them.
+ * Children of a {@link org.apache.calcite.plan.RelOptRuleOperand} and the
+ * policy for matching them.
  *
  * <p>Often created by calling one of the following methods:
  * {@link RelOptRule#some},

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/plan/RelOptSamplingParameters.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/RelOptSamplingParameters.java b/core/src/main/java/org/apache/calcite/plan/RelOptSamplingParameters.java
index 89f54d6..7a50613 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptSamplingParameters.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptSamplingParameters.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.relopt;
+package org.apache.calcite.plan;
 
 /**
  * RelOptSamplingParameters represents the parameters necessary to produce a
@@ -74,8 +74,8 @@ public class RelOptSamplingParameters {
    * Indicates whether the sample results should be repeatable. Sample results
    * are only required to repeat if no changes have been made to the
    * relation's content or structure. If the sample is configured to be
-   * repeatable, then a user-specified seed value can be obtained via {@link
-   * #getRepeatableSeed()}.
+   * repeatable, then a user-specified seed value can be obtained via
+   * {@link #getRepeatableSeed()}.
    *
    * @return true if the sample results should be repeatable
    */

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/plan/RelOptSchema.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/RelOptSchema.java b/core/src/main/java/org/apache/calcite/plan/RelOptSchema.java
index 489a40b..1b682ff 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptSchema.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptSchema.java
@@ -14,11 +14,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.relopt;
+package org.apache.calcite.plan;
 
-import java.util.List;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
 
-import org.eigenbase.reltype.*;
+import java.util.List;
 
 /**
  * A <code>RelOptSchema</code> is a set of {@link RelOptTable} objects.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/plan/RelOptSchemaWithSampling.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/RelOptSchemaWithSampling.java b/core/src/main/java/org/apache/calcite/plan/RelOptSchemaWithSampling.java
index 93a281d..77f6716 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptSchemaWithSampling.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptSchemaWithSampling.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.relopt;
+package org.apache.calcite.plan;
 
 import java.util.List;
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/plan/RelOptTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/RelOptTable.java b/core/src/main/java/org/apache/calcite/plan/RelOptTable.java
index bd0fdbc..571fbd1 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptTable.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptTable.java
@@ -14,14 +14,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.relopt;
+package org.apache.calcite.plan;
 
-import java.util.*;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.type.RelDataType;
 
-import org.eigenbase.rel.*;
-import org.eigenbase.reltype.*;
-
-import net.hydromatic.linq4j.expressions.Expression;
+import java.util.BitSet;
+import java.util.List;
 
 /**
  * Represents a relational dataset in a {@link RelOptSchema}. It has methods to
@@ -56,11 +57,12 @@ public interface RelOptTable {
   /**
    * Converts this table into a {@link RelNode relational expression}.
    *
-   * <p>The {@link org.eigenbase.relopt.RelOptPlanner planner} calls this
+   * <p>The {@link org.apache.calcite.plan.RelOptPlanner planner} calls this
    * method to convert a table into an initial relational expression,
-   * generally something abstract, such as a {@link
-   * org.eigenbase.rel.TableAccessRel}, then optimizes this expression by
-   * applying {@link org.eigenbase.relopt.RelOptRule rules} to transform it
+   * generally something abstract, such as a
+   * {@link org.apache.calcite.rel.logical.LogicalTableScan},
+   * then optimizes this expression by
+   * applying {@link org.apache.calcite.plan.RelOptRule rules} to transform it
    * into more efficient access methods for this table.</p>
    */
   RelNode toRel(ToRelContext context);

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 ec81f86..e188d35 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
@@ -14,30 +14,85 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.relopt;
-
-import java.io.*;
-import java.util.*;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.rel.rules.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.rex.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.fun.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.sql.validate.SqlValidatorUtil;
-import org.eigenbase.util.*;
-import org.eigenbase.util.mapping.*;
-
-import net.hydromatic.linq4j.Ord;
-
-import net.hydromatic.optiq.util.BitSets;
+package org.apache.calcite.plan;
+
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollationImpl;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelVisitor;
+import org.apache.calcite.rel.RelWriter;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.RelFactories;
+import org.apache.calcite.rel.core.SemiJoin;
+import org.apache.calcite.rel.externalize.RelWriterImpl;
+import org.apache.calcite.rel.externalize.RelXmlWriter;
+import org.apache.calcite.rel.logical.LogicalAggregate;
+import org.apache.calcite.rel.logical.LogicalCalc;
+import org.apache.calcite.rel.logical.LogicalFilter;
+import org.apache.calcite.rel.logical.LogicalJoin;
+import org.apache.calcite.rel.logical.LogicalProject;
+import org.apache.calcite.rel.rules.AggregateProjectPullUpConstantsRule;
+import org.apache.calcite.rel.rules.EmptyPruneRules;
+import org.apache.calcite.rel.rules.FilterMergeRule;
+import org.apache.calcite.rel.rules.MultiJoin;
+import org.apache.calcite.rel.rules.ProjectRemoveRule;
+import org.apache.calcite.rel.rules.ProjectToWindowRule;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rel.type.RelDataTypeFieldImpl;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexCorrelVariable;
+import org.apache.calcite.rex.RexFieldAccess;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexLocalRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexProgramBuilder;
+import org.apache.calcite.rex.RexShuttle;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.sql.SqlAggFunction;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.fun.SqlMinMaxAggFunction;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.type.MultisetSqlType;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.validate.SqlValidatorUtil;
+import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.Bug;
+import org.apache.calcite.util.Holder;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Permutation;
+import org.apache.calcite.util.Util;
+import org.apache.calcite.util.mapping.Mapping;
+import org.apache.calcite.util.mapping.MappingType;
+import org.apache.calcite.util.mapping.Mappings;
 
 import com.google.common.base.Function;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.util.AbstractList;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashSet;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+
 /**
  * <code>RelOptUtil</code> defines static utility methods for use in optimizing
  * {@link RelNode}s.
@@ -93,7 +148,7 @@ public abstract class RelOptUtil {
   /**
    * Returns a set of variables used by a relational expression or its
    * descendants. The set may contain duplicates. The item type is the same as
-   * {@link org.eigenbase.rex.RexVariable#getName}
+   * {@link org.apache.calcite.rex.RexVariable#getName}
    */
   public static Set<String> getVariablesUsed(RelNode rel) {
     final VariableUsedVisitor vuv = new VariableUsedVisitor();
@@ -136,7 +191,7 @@ public abstract class RelOptUtil {
    *
    * @param type Struct type
    * @return List of field types
-   * @see org.eigenbase.reltype.RelDataType#getFieldNames()
+   * @see org.apache.calcite.rel.type.RelDataType#getFieldNames()
    */
   public static List<RelDataType> getFieldTypeList(final RelDataType type) {
     return Lists.transform(type.getFieldList(), GET_TYPE);
@@ -194,12 +249,9 @@ public abstract class RelOptUtil {
       return;
     }
 
-    String s =
-        "Cannot add expression of different type to set:\n"
-        + "set type is "
-        + expectedRowType.getFullTypeString()
-        + "\nexpression type is "
-        + actualRowType.getFullTypeString()
+    String s = "Cannot add expression of different type to set:\n"
+        + "set type is " + expectedRowType.getFullTypeString()
+        + "\nexpression type is " + actualRowType.getFullTypeString()
         + "\nset is " + equivalenceClass.toString()
         + "\nexpression is " + newRel.toString();
     throw Util.newInternal(s);
@@ -238,11 +290,13 @@ public abstract class RelOptUtil {
 
   /**
    * Creates a plan suitable for use in <code>EXISTS</code> or <code>IN</code>
-   * statements. See {@link
-   * org.eigenbase.sql2rel.SqlToRelConverter#convertExists} Note: this
-   * implementation of createExistsPlan is only called from
-   * net.sf.farrago.fennel.rel. The last two arguments do not apply to
-   * those invocations and can be removed from the method.
+   * statements.
+   *
+   * <p>See {@link org.apache.calcite.sql2rel.SqlToRelConverter#convertExists}
+   *
+   * <p>Note: this implementation of createExistsPlan is only called from
+   * net.sf.farrago.fennel.rel. The last two arguments do not apply to those
+   * invocations and can be removed from the method.
    *
    * @param cluster    Cluster
    * @param seekRel    A query rel, for example the resulting rel from 'select *
@@ -303,7 +357,7 @@ public abstract class RelOptUtil {
               extraName);
 
       ret =
-          new AggregateRel(
+          new LogicalAggregate(
               ret.getCluster(),
               ret,
               BitSets.of(),
@@ -317,7 +371,7 @@ public abstract class RelOptUtil {
    * Creates a plan suitable for use in <code>EXISTS</code> or <code>IN</code>
    * statements.
    *
-   * @see org.eigenbase.sql2rel.SqlToRelConverter#convertExists
+   * @see org.apache.calcite.sql2rel.SqlToRelConverter#convertExists
    *
    * @param seekRel    A query rel, for example the resulting rel from 'select *
    *                   from emp' or 'values (1,2,3)' or '('Foo', 34)'.
@@ -347,7 +401,7 @@ public abstract class RelOptUtil {
       final int keyCount = ret.getRowType().getFieldCount();
       if (!needsOuterJoin) {
         return Pair.<RelNode, Boolean>of(
-            new AggregateRel(cluster, ret, BitSets.range(keyCount),
+            new LogicalAggregate(cluster, ret, BitSets.range(keyCount),
                 ImmutableList.<AggregateCall>of()),
             false);
       }
@@ -381,7 +435,7 @@ public abstract class RelOptUtil {
               null,
               null);
 
-      ret = new AggregateRel(
+      ret = new LogicalAggregate(
           cluster,
           ret,
           BitSets.range(projectedKeyCount),
@@ -398,10 +452,10 @@ public abstract class RelOptUtil {
   }
 
   /**
-   * Creates a ProjectRel which accomplishes a rename.
+   * Creates a LogicalProject which accomplishes a rename.
    *
    * @param outputType a row type descriptor whose field names the generated
-   *                   ProjectRel must match
+   *                   LogicalProject must match
    * @param rel        the rel whose output is to be renamed; rel.getRowType()
    *                   must be the same as outputType except for field names
    * @return generated relational expression
@@ -414,8 +468,8 @@ public abstract class RelOptUtil {
     int n = inputFields.size();
 
     List<RelDataTypeField> outputFields = outputType.getFieldList();
-    assert outputFields.size() == n : "rename: field count mismatch: in="
-        + inputType
+    assert outputFields.size() == n
+        : "rename: field count mismatch: in=" + inputType
         + ", out" + outputType;
 
     List<Pair<RexNode, String>> renames =
@@ -597,8 +651,8 @@ public abstract class RelOptUtil {
   }
 
   /**
-   * Creates an AggregateRel which removes all duplicates from the result of
-   * an underlying rel.
+   * Creates a LogicalAggregate that removes all duplicates from the result of
+   * an underlying relational expression.
    *
    * @param rel underlying rel
    * @return rel implementing SingleValueAgg
@@ -626,7 +680,7 @@ public abstract class RelOptUtil {
               null));
     }
 
-    return new AggregateRel(
+    return new LogicalAggregate(
         rel.getCluster(),
         rel,
         BitSets.of(),
@@ -634,15 +688,14 @@ public abstract class RelOptUtil {
   }
 
   /**
-   * Creates an AggregateRel which removes all duplicates from the result of
-   * an underlying rel.
+   * Creates a LogicalAggregate that removes all duplicates from the result of
+   * an underlying relational expression.
    *
    * @param rel underlying rel
    * @return rel implementing DISTINCT
    */
-  public static RelNode createDistinctRel(
-      RelNode rel) {
-    return new AggregateRel(
+  public static RelNode createDistinctRel(RelNode rel) {
+    return new LogicalAggregate(
         rel.getCluster(),
         rel,
         BitSets.range(rel.getRowType().getFieldCount()),
@@ -650,9 +703,9 @@ public abstract class RelOptUtil {
   }
 
   public static boolean analyzeSimpleEquiJoin(
-      JoinRel joinRel,
+      LogicalJoin join,
       int[] joinFieldOrdinals) {
-    RexNode joinExp = joinRel.getCondition();
+    RexNode joinExp = join.getCondition();
     if (joinExp.getKind() != SqlKind.EQUALS) {
       return false;
     }
@@ -667,7 +720,7 @@ public abstract class RelOptUtil {
     }
 
     final int leftFieldCount =
-        joinRel.getLeft().getRowType().getFieldCount();
+        join.getLeft().getRowType().getFieldCount();
     RexInputRef leftFieldAccess = (RexInputRef) leftComparand;
     if (!(leftFieldAccess.getIndex() < leftFieldCount)) {
       // left field must access left side of join
@@ -805,33 +858,33 @@ public abstract class RelOptUtil {
   }
 
   public static RexNode splitCorrelatedFilterCondition(
-      FilterRel filterRel,
+      LogicalFilter filter,
       List<RexInputRef> joinKeys,
       List<RexNode> correlatedJoinKeys) {
     List<RexNode> nonEquiList = new ArrayList<RexNode>();
 
     splitCorrelatedFilterCondition(
-        filterRel,
-        filterRel.getCondition(),
+        filter,
+        filter.getCondition(),
         joinKeys,
         correlatedJoinKeys,
         nonEquiList);
 
     // Convert the remainders into a list that are AND'ed together.
     return RexUtil.composeConjunction(
-        filterRel.getCluster().getRexBuilder(), nonEquiList, true);
+        filter.getCluster().getRexBuilder(), nonEquiList, true);
   }
 
   public static RexNode splitCorrelatedFilterCondition(
-      FilterRel filterRel,
+      LogicalFilter filter,
       List<RexNode> joinKeys,
       List<RexNode> correlatedJoinKeys,
       boolean extractCorrelatedFieldAccess) {
     List<RexNode> nonEquiList = new ArrayList<RexNode>();
 
     splitCorrelatedFilterCondition(
-        filterRel,
-        filterRel.getCondition(),
+        filter,
+        filter.getCondition(),
         joinKeys,
         correlatedJoinKeys,
         nonEquiList,
@@ -839,7 +892,7 @@ public abstract class RelOptUtil {
 
     // Convert the remainders into a list that are AND'ed together.
     return RexUtil.composeConjunction(
-        filterRel.getCluster().getRexBuilder(), nonEquiList, true);
+        filter.getCluster().getRexBuilder(), nonEquiList, true);
   }
 
   private static void splitJoinCondition(
@@ -1136,7 +1189,7 @@ public abstract class RelOptUtil {
   }
 
   private static void splitCorrelatedFilterCondition(
-      FilterRel filterRel,
+      LogicalFilter filter,
       RexNode condition,
       List<RexInputRef> joinKeys,
       List<RexNode> correlatedJoinKeys,
@@ -1146,7 +1199,7 @@ public abstract class RelOptUtil {
       if (call.getOperator() == SqlStdOperatorTable.AND) {
         for (RexNode operand : call.getOperands()) {
           splitCorrelatedFilterCondition(
-              filterRel,
+              filter,
               operand,
               joinKeys,
               correlatedJoinKeys,
@@ -1182,7 +1235,7 @@ public abstract class RelOptUtil {
   }
 
   private static void splitCorrelatedFilterCondition(
-      FilterRel filterRel,
+      LogicalFilter filter,
       RexNode condition,
       List<RexNode> joinKeys,
       List<RexNode> correlatedJoinKeys,
@@ -1193,7 +1246,7 @@ public abstract class RelOptUtil {
       if (call.getOperator() == SqlStdOperatorTable.AND) {
         for (RexNode operand : call.getOperands()) {
           splitCorrelatedFilterCondition(
-              filterRel,
+              filter,
               operand,
               joinKeys,
               correlatedJoinKeys,
@@ -1464,17 +1517,17 @@ public abstract class RelOptUtil {
   }
 
   public static void registerAbstractRels(RelOptPlanner planner) {
-    planner.addRule(PullConstantsThroughAggregatesRule.INSTANCE);
-    planner.addRule(RemoveEmptyRules.UNION_INSTANCE);
-    planner.addRule(RemoveEmptyRules.PROJECT_INSTANCE);
-    planner.addRule(RemoveEmptyRules.FILTER_INSTANCE);
-    planner.addRule(RemoveEmptyRules.SORT_INSTANCE);
-    planner.addRule(RemoveEmptyRules.AGGREGATE_INSTANCE);
-    planner.addRule(RemoveEmptyRules.JOIN_LEFT_INSTANCE);
-    planner.addRule(RemoveEmptyRules.JOIN_RIGHT_INSTANCE);
-    planner.addRule(RemoveEmptyRules.SORT_FETCH_ZERO_INSTANCE);
-    planner.addRule(WindowedAggSplitterRule.PROJECT);
-    planner.addRule(MergeFilterRule.INSTANCE);
+    planner.addRule(AggregateProjectPullUpConstantsRule.INSTANCE);
+    planner.addRule(EmptyPruneRules.UNION_INSTANCE);
+    planner.addRule(EmptyPruneRules.PROJECT_INSTANCE);
+    planner.addRule(EmptyPruneRules.FILTER_INSTANCE);
+    planner.addRule(EmptyPruneRules.SORT_INSTANCE);
+    planner.addRule(EmptyPruneRules.AGGREGATE_INSTANCE);
+    planner.addRule(EmptyPruneRules.JOIN_LEFT_INSTANCE);
+    planner.addRule(EmptyPruneRules.JOIN_RIGHT_INSTANCE);
+    planner.addRule(EmptyPruneRules.SORT_FETCH_ZERO_INSTANCE);
+    planner.addRule(ProjectToWindowRule.PROJECT);
+    planner.addRule(FilterMergeRule.INSTANCE);
   }
 
   /**
@@ -1574,9 +1627,9 @@ public abstract class RelOptUtil {
   }
 
   /**
-   * Returns whether two types are equal using {@link
-   * #areRowTypesEqual(RelDataType, RelDataType, boolean)}. Both types must
-   * not be null.
+   * Returns whether two types are equal using
+   * {@link #areRowTypesEqual(RelDataType, RelDataType, boolean)}. Both types
+   * must not be null.
    *
    * @param desc1 Description of role of first type
    * @param type1 First type
@@ -1891,7 +1944,7 @@ public abstract class RelOptUtil {
    * @param right      filter on the right
    * @return AND'd filter
    *
-   * @see org.eigenbase.rex.RexUtil#composeConjunction
+   * @see org.apache.calcite.rex.RexUtil#composeConjunction
    */
   public static RexNode andJoinFilters(
       RexBuilder rexBuilder,
@@ -1995,7 +2048,7 @@ public abstract class RelOptUtil {
    *
    * @deprecated Use the other {@link #classifyFilters};
    * very short-term; will be removed before
-   * {@link org.eigenbase.util.Bug#upgrade(String) calcite-0.9.2}.
+   * {@link org.apache.calcite.util.Bug#upgrade(String) calcite-0.9.2}.
    */
   public static boolean classifyFilters(
       RelNode joinRel,
@@ -2059,7 +2112,7 @@ public abstract class RelOptUtil {
     final List<RelDataTypeField> rightFields =
         joinRel.getInputs().get(1).getRowType().getFieldList();
     final int nFieldsRight = rightFields.size();
-    assert nTotalFields == (joinRel instanceof SemiJoinRel
+    assert nTotalFields == (joinRel instanceof SemiJoin
         ? nSysFields + nFieldsLeft
         : nSysFields + nFieldsLeft + nFieldsRight);
 
@@ -2266,14 +2319,14 @@ public abstract class RelOptUtil {
    * the input references are the same but the field names are different
    */
   public static boolean checkProjAndChildInputs(
-      ProjectRelBase project,
+      Project project,
       boolean checkNames) {
     if (!project.isBoxed()) {
       return false;
     }
 
     int n = project.getProjects().size();
-    RelDataType inputType = project.getChild().getRowType();
+    RelDataType inputType = project.getInput().getRowType();
     if (inputType.getFieldList().size() != n) {
       return false;
     }
@@ -2309,7 +2362,7 @@ public abstract class RelOptUtil {
    *
    * @param newJoin   the RelNode corresponding to the join with its inputs
    *                  swapped
-   * @param origJoin  original JoinRel
+   * @param origJoin  original LogicalJoin
    * @param origOrder if true, create the projection expressions to reflect
    *                  the original (pre-swapped) join projection; otherwise,
    *                  create the projection to reflect the order of the swapped
@@ -2318,7 +2371,7 @@ public abstract class RelOptUtil {
    */
   public static List<RexNode> createSwappedJoinExprs(
       RelNode newJoin,
-      JoinRelBase origJoin,
+      Join origJoin,
       boolean origOrder) {
     final List<RelDataTypeField> newJoinFields =
         newJoin.getRowType().getFieldList();
@@ -2338,7 +2391,7 @@ public abstract class RelOptUtil {
 
   /**
    * Converts a filter to the new filter that would result if the filter is
-   * pushed past a ProjectRel that it currently is referencing.
+   * pushed past a LogicalProject that it currently is referencing.
    *
    * @param filter  the filter to be converted
    * @param projRel project rel underneath the filter
@@ -2346,14 +2399,14 @@ public abstract class RelOptUtil {
    */
   public static RexNode pushFilterPastProject(
       RexNode filter,
-      ProjectRelBase projRel) {
-    // use RexPrograms to merge the filter and ProjectRel into a
-    // single program so we can convert the FilterRel condition to
-    // directly reference the ProjectRel's child
+      Project projRel) {
+    // use RexPrograms to merge the filter and LogicalProject into a
+    // single program so we can convert the LogicalFilter condition to
+    // directly reference the LogicalProject's child
     RexBuilder rexBuilder = projRel.getCluster().getRexBuilder();
     RexProgram bottomProgram =
         RexProgram.create(
-            projRel.getChild().getRowType(),
+            projRel.getInput().getRowType(),
             projRel.getProjects(),
             null,
             projRel.getRowType(),
@@ -2378,19 +2431,21 @@ public abstract class RelOptUtil {
   }
 
   /**
-   * Creates a new {@link MultiJoinRel} to reflect projection references from
-   * a {@link ProjectRel} that is on top of the {@link MultiJoinRel}.
-   *
-   * @param multiJoin the original MultiJoinRel
-   * @param project   the ProjectRel on top of the MultiJoinRel
-   * @return the new MultiJoinRel
+   * Creates a new {@link org.apache.calcite.rel.rules.MultiJoin} to reflect
+   * projection references from a
+   * {@link org.apache.calcite.rel.logical.LogicalProject} that is on top of the
+   * {@link org.apache.calcite.rel.rules.MultiJoin}.
+   *
+   * @param multiJoin the original MultiJoin
+   * @param project   the LogicalProject on top of the MultiJoin
+   * @return the new MultiJoin
    */
-  public static MultiJoinRel projectMultiJoin(
-      MultiJoinRel multiJoin,
-      ProjectRel project) {
+  public static MultiJoin projectMultiJoin(
+      MultiJoin multiJoin,
+      LogicalProject project) {
     // Locate all input references in the projection expressions as well
     // the post-join filter.  Since the filter effectively sits in
-    // between the ProjectRel and the MultiJoinRel, the projection needs
+    // between the LogicalProject and the MultiJoin, the projection needs
     // to include those filter references.
     BitSet inputRefs = InputFinder.bits(
         project.getProjects(), multiJoin.getPostJoinFilter());
@@ -2418,9 +2473,9 @@ public abstract class RelOptUtil {
       newProjFields.get(currInput).set(bit - startField);
     }
 
-    // create a new MultiJoinRel containing the new field bitmaps
+    // create a new MultiJoin containing the new field bitmaps
     // for each input
-    return new MultiJoinRel(
+    return new MultiJoin(
         multiJoin.getCluster(),
         multiJoin.getInputs(),
         multiJoin.getJoinFilter(),
@@ -2456,10 +2511,10 @@ public abstract class RelOptUtil {
   }
 
   /**
-   * Creates a {@link org.eigenbase.rel.ProjectRel} that projects particular
-   * fields of its input, according to a mapping.
+   * Creates a {@link org.apache.calcite.rel.logical.LogicalProject} that
+   * projects particular fields of its input, according to a mapping.
    */
-  public static ProjectRel project(
+  public static LogicalProject project(
       RelNode child,
       Mappings.TargetMapping mapping) {
     List<RexNode> nodes = new ArrayList<RexNode>();
@@ -2471,8 +2526,8 @@ public abstract class RelOptUtil {
       nodes.add(new RexInputRef(source, field.getType()));
       names.add(field.getName());
     }
-    return new ProjectRel(
-        child.getCluster(), child, nodes, names, ProjectRel.Flags.BOXED);
+    return new LogicalProject(
+        child.getCluster(), child, nodes, names, LogicalProject.Flags.BOXED);
   }
 
   /** Returns whether relational expression {@code target} occurs within a
@@ -2532,7 +2587,8 @@ public abstract class RelOptUtil {
     return query;
   }
 
-  /** Returns a simple {@link org.eigenbase.relopt.RelOptTable.ToRelContext}. */
+  /** Returns a simple
+   * {@link org.apache.calcite.plan.RelOptTable.ToRelContext}. */
   public static RelOptTable.ToRelContext getContext(
       final RelOptCluster cluster) {
     return new RelOptTable.ToRelContext() {
@@ -2549,7 +2605,7 @@ public abstract class RelOptUtil {
     };
   }
 
-  /** Returns the number of {@link org.eigenbase.rel.JoinRelBase} nodes in a
+  /** Returns the number of {@link org.apache.calcite.rel.core.Join} nodes in a
    * tree. */
   public static int countJoins(RelNode rootRel) {
     /** Visitor that counts join nodes. */
@@ -2557,7 +2613,7 @@ public abstract class RelOptUtil {
       int joinCount;
 
       @Override public void visit(RelNode node, int ordinal, RelNode parent) {
-        if (node instanceof JoinRelBase) {
+        if (node instanceof Join) {
           ++joinCount;
         }
         super.visit(node, ordinal, parent);
@@ -2629,10 +2685,11 @@ public abstract class RelOptUtil {
    * Creates a relational expression which projects an array of expressions,
    * and optionally optimizes.
    *
-   * <p>The result may not be a {@link org.eigenbase.rel.ProjectRel}. If the
+   * <p>The result may not be a
+   * {@link org.apache.calcite.rel.logical.LogicalProject}. If the
    * projection is trivial, <code>child</code> is returned directly; and future
    * versions may return other formulations of expressions, such as
-   * {@link org.eigenbase.rel.CalcRel}.
+   * {@link org.apache.calcite.rel.logical.LogicalCalc}.
    *
    * @param child      input relational expression
    * @param exprs      list of expressions for the input columns
@@ -2661,18 +2718,17 @@ public abstract class RelOptUtil {
                 : SqlValidatorUtil.uniquify(
                     fieldNames, SqlValidatorUtil.F_SUGGESTER));
     if (optimize
-        && RemoveTrivialProjectRule.isIdentity(exprs, rowType,
-            child.getRowType())) {
+        && ProjectRemoveRule.isIdentity(exprs, rowType, child.getRowType())) {
       return child;
     }
-    return new ProjectRel(cluster,
+    return new LogicalProject(cluster,
         cluster.traitSetOf(collationList.isEmpty()
             ? RelCollationImpl.EMPTY
             : collationList.get(0)),
         child,
         exprs,
         rowType,
-        ProjectRelBase.Flags.BOXED);
+        Project.Flags.BOXED);
   }
 
   /**
@@ -2708,9 +2764,10 @@ public abstract class RelOptUtil {
    * <p>Optimizations:</p>
    *
    * <ul>
-   * <li>If the relational expression is a {@link org.eigenbase.rel.CalcRel} or
-   * {@link org.eigenbase.rel.ProjectRel} that is already acting as a
-   * permutation, combines the new permutation with the old;</li>
+   * <li>If the relational expression is a
+   * {@link org.apache.calcite.rel.logical.LogicalCalc} or
+   * {@link org.apache.calcite.rel.logical.LogicalProject} that is already
+   * acting as a permutation, combines the new permutation with the old;</li>
    *
    * <li>If the permutation is the identity, returns the original relational
    * expression.</li>
@@ -2732,16 +2789,16 @@ public abstract class RelOptUtil {
     if (permutation.isIdentity()) {
       return rel;
     }
-    if (rel instanceof CalcRel) {
-      CalcRel calcRel = (CalcRel) rel;
-      Permutation permutation1 = calcRel.getProgram().getPermutation();
+    if (rel instanceof LogicalCalc) {
+      LogicalCalc calc = (LogicalCalc) rel;
+      Permutation permutation1 = calc.getProgram().getPermutation();
       if (permutation1 != null) {
         Permutation permutation2 = permutation.product(permutation1);
         return permute(rel, permutation2, null);
       }
     }
-    if (rel instanceof ProjectRel) {
-      Permutation permutation1 = ((ProjectRel) rel).getPermutation();
+    if (rel instanceof LogicalProject) {
+      Permutation permutation1 = ((LogicalProject) rel).getPermutation();
       if (permutation1 != null) {
         Permutation permutation2 = permutation.product(permutation1);
         return permute(rel, permutation2, null);
@@ -2780,7 +2837,7 @@ public abstract class RelOptUtil {
             rel.getCluster().getTypeFactory().createStructType(
                 outputTypeList,
                 outputNameList));
-    return new CalcRel(
+    return new LogicalCalc(
         rel.getCluster(),
         rel.getTraitSet(),
         rel,
@@ -2836,13 +2893,13 @@ public abstract class RelOptUtil {
    * does not at present.
    *
    * @param rel        Relational expression
-   * @param mapping    Mapping from source fields to target fields. The mapping
-   *                   type must obey the constraints
-   *                   {@link org.eigenbase.util.mapping.MappingType#isMandatorySource()}
-   *                   and
-   *                   {@link org.eigenbase.util.mapping.MappingType#isSingleSource()},
-   *                   as does
-   *                   {@link org.eigenbase.util.mapping.MappingType#INVERSE_FUNCTION}.
+   * @param mapping Mapping from source fields to target fields. The mapping
+   * type must obey the constraints
+   * {@link org.apache.calcite.util.mapping.MappingType#isMandatorySource()}
+   * and
+   * {@link org.apache.calcite.util.mapping.MappingType#isSingleSource()},
+   * as does
+   * {@link org.apache.calcite.util.mapping.MappingType#INVERSE_FUNCTION}.
    * @param fieldNames Field names; if null, or if a particular entry is null,
    *                   the name of the permuted field is used
    * @return relational expression which projects a subset of the input fields
@@ -3056,8 +3113,7 @@ public abstract class RelOptUtil {
       return null;
     }
 
-    @Override
-    public Void visitCall(RexCall call) {
+    @Override public Void visitCall(RexCall call) {
       if (call.getOperator() == RexBuilder.GET_OPERATOR) {
         RexLiteral literal = (RexLiteral) call.getOperands().get(1);
         extraFields.add(

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/plan/RelTrait.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/RelTrait.java b/core/src/main/java/org/apache/calcite/plan/RelTrait.java
index 4baf877..a3c3efe 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelTrait.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelTrait.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.relopt;
+package org.apache.calcite.plan;
 
 /**
  * RelTrait represents the manifestation of a relational expression trait within
@@ -27,9 +27,9 @@ package org.eigenbase.relopt;
  * an {@code enum} and no new RelTraits can be introduced at runtime, you need
  * not override {@link #hashCode()} and {@link #equals(Object)}. If, however,
  * new RelTrait instances are generated at runtime (e.g. based on state external
- * to the planner), you must implement {@link #hashCode()} and {@link
- * #equals(Object)} for proper {@link RelTraitDef#canonize canonization} of your
- * RelTrait objects.</p>
+ * to the planner), you must implement {@link #hashCode()} and
+ * {@link #equals(Object)} for proper {@link RelTraitDef#canonize canonization}
+ * of your RelTrait objects.</p>
  */
 public interface RelTrait {
   //~ Methods ----------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/plan/RelTraitDef.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/RelTraitDef.java b/core/src/main/java/org/apache/calcite/plan/RelTraitDef.java
index f26bcc0..0273568 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelTraitDef.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelTraitDef.java
@@ -14,12 +14,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.relopt;
+package org.apache.calcite.plan;
 
-import org.eigenbase.rel.*;
-import org.eigenbase.rel.convert.*;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
 
-import com.google.common.cache.*;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
 
 /**
  * RelTraitDef represents a class of {@link RelTrait}s. Implementations of
@@ -33,9 +35,10 @@ import com.google.common.cache.*;
  * <li>Either
  *
  * <ul>
- * <li> {@link #canConvert(RelOptPlanner, RelTrait, RelTrait)} and {@link
- * #convert(RelOptPlanner, RelNode, RelTrait, boolean)} do not require
+ * <li> {@link #canConvert(RelOptPlanner, RelTrait, RelTrait)} and
+ * {@link #convert(RelOptPlanner, RelNode, RelTrait, boolean)} do not require
  * planner-instance-specific information, <b>or</b></li>
+ *
  * <li>the RelTraitDef manages separate sets of conversion data internally. See
  * {@link ConventionTraitDef} for an example of this.</li>
  * </ul>
@@ -55,8 +58,7 @@ public abstract class RelTraitDef<T extends RelTrait> {
           .softValues()
           .build(
               new CacheLoader<T, T>() {
-                @Override
-                public T load(T key) throws Exception {
+                @Override public T load(T key) throws Exception {
                   return key;
                 }
               });
@@ -85,7 +87,7 @@ public abstract class RelTraitDef<T extends RelTrait> {
 
   /**
    * @return a simple name for this RelTraitDef (for use in
-   * {@link org.eigenbase.rel.RelNode#explain(RelWriter)}).
+   * {@link org.apache.calcite.rel.RelNode#explain}).
    */
   public abstract String getSimpleName();
 
@@ -140,9 +142,9 @@ public abstract class RelTraitDef<T extends RelTrait> {
       T toTrait);
 
   /**
-   * Provides notification of the registration of a particular {@link
-   * ConverterRule} with a {@link RelOptPlanner}. The default implementation
-   * does nothing.
+   * Provides notification of the registration of a particular
+   * {@link ConverterRule} with a {@link RelOptPlanner}. The default
+   * implementation does nothing.
    *
    * @param planner       the planner registering the rule
    * @param converterRule the registered converter rule

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/plan/RelTraitPropagationVisitor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/RelTraitPropagationVisitor.java b/core/src/main/java/org/apache/calcite/plan/RelTraitPropagationVisitor.java
index ec6b24f..486fecc 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelTraitPropagationVisitor.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelTraitPropagationVisitor.java
@@ -14,10 +14,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.relopt;
+package org.apache.calcite.plan;
 
-import org.eigenbase.rel.*;
-import org.eigenbase.util.Util;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelVisitor;
+import org.apache.calcite.util.Util;
 
 /**
  * RelTraitPropagationVisitor traverses a RelNode and its <i>unregistered</i>

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/plan/RelTraitSet.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/RelTraitSet.java b/core/src/main/java/org/apache/calcite/plan/RelTraitSet.java
index ecd2ec0..33072d1 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelTraitSet.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelTraitSet.java
@@ -14,16 +14,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.relopt;
+package org.apache.calcite.plan;
 
-import java.util.*;
-
-import org.eigenbase.util.Pair;
-
-import net.hydromatic.optiq.runtime.FlatLists;
+import org.apache.calcite.runtime.FlatLists;
+import org.apache.calcite.util.Pair;
 
 import com.google.common.collect.ImmutableList;
 
+import java.util.AbstractList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
 /**
  * RelTraitSet represents an ordered set of {@link RelTrait}s.
  */
@@ -173,15 +176,13 @@ public final class RelTraitSet extends AbstractList<RelTrait> {
    * @param obj another RelTraitSet
    * @return true if traits are equal and in the same order, false otherwise
    */
-  @Override
-  public boolean equals(Object obj) {
+  @Override public boolean equals(Object obj) {
     return this == obj
         || obj instanceof RelTraitSet
         && Arrays.equals(traits, ((RelTraitSet) obj).traits);
   }
 
-  @Override
-  public int hashCode() {
+  @Override public int hashCode() {
     return Arrays.hashCode(traits);
   }
 
@@ -284,8 +285,7 @@ public final class RelTraitSet extends AbstractList<RelTrait> {
     return Arrays.equals(traits, relTraits);
   }
 
-  @Override
-  public String toString() {
+  @Override public String toString() {
     return string;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/plan/Strong.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/Strong.java b/core/src/main/java/org/apache/calcite/plan/Strong.java
index 4415757..e279b14 100644
--- a/core/src/main/java/org/apache/calcite/plan/Strong.java
+++ b/core/src/main/java/org/apache/calcite/plan/Strong.java
@@ -14,16 +14,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.relopt;
+package org.apache.calcite.plan;
+
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
 
 import java.util.BitSet;
 import java.util.List;
 
-import org.eigenbase.rex.RexCall;
-import org.eigenbase.rex.RexInputRef;
-import org.eigenbase.rex.RexLiteral;
-import org.eigenbase.rex.RexNode;
-
 /** Utilities for strong predicates.
  *
  * <p>A predicate is strong (or null-rejecting) if it is UNKNOWN if any of its


[31/58] [abbrv] [partial] incubator-calcite git commit: [CALCITE-306] Standardize code style for "import package.*; "

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/rules/ProjectToCalcRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/ProjectToCalcRule.java b/core/src/main/java/org/apache/calcite/rel/rules/ProjectToCalcRule.java
index 0e921c8..c6fc5fe 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/ProjectToCalcRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/ProjectToCalcRule.java
@@ -14,21 +14,30 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
+package org.apache.calcite.rel.rules;
 
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.rex.*;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.logical.LogicalCalc;
+import org.apache.calcite.rel.logical.LogicalProject;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexProgram;
 
 import com.google.common.collect.ImmutableList;
 
 /**
- * Rule to convert a {@link ProjectRel} to a {@link CalcRel}
+ * Rule to convert a
+ * {@link org.apache.calcite.rel.logical.LogicalProject} to a
+ * {@link org.apache.calcite.rel.logical.LogicalCalc}
  *
- * <p>The rule does not fire if the child is a {@link ProjectRel}, {@link
- * FilterRel} or {@link CalcRel}. If it did, then the same {@link CalcRel} would
- * be formed via several transformation paths, which is a waste of effort.</p>
+ * <p>The rule does not fire if the child is a
+ * {@link org.apache.calcite.rel.logical.LogicalProject},
+ * {@link org.apache.calcite.rel.logical.LogicalFilter} or
+ * {@link org.apache.calcite.rel.logical.LogicalCalc}. If it did, then the same
+ * {@link org.apache.calcite.rel.logical.LogicalCalc} would be formed via
+ * several transformation paths, which is a waste of effort.</p>
  *
  * @see FilterToCalcRule
  */
@@ -40,14 +49,14 @@ public class ProjectToCalcRule extends RelOptRule {
   //~ Constructors -----------------------------------------------------------
 
   private ProjectToCalcRule() {
-    super(operand(ProjectRel.class, any()));
+    super(operand(LogicalProject.class, any()));
   }
 
   //~ Methods ----------------------------------------------------------------
 
   public void onMatch(RelOptRuleCall call) {
-    final ProjectRel project = call.rel(0);
-    final RelNode child = project.getChild();
+    final LogicalProject project = call.rel(0);
+    final RelNode child = project.getInput();
     final RelDataType rowType = project.getRowType();
     final RexProgram program =
         RexProgram.create(
@@ -56,8 +65,8 @@ public class ProjectToCalcRule extends RelOptRule {
             null,
             project.getRowType(),
             project.getCluster().getRexBuilder());
-    final CalcRel calc =
-        new CalcRel(
+    final LogicalCalc calc =
+        new LogicalCalc(
             project.getCluster(),
             project.getTraitSet(),
             child,

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 ac69d6f..b3a8023 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
@@ -14,57 +14,85 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
+package org.apache.calcite.rel.rules;
 
-import java.util.*;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.rex.*;
-import org.eigenbase.util.*;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptRuleOperand;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Calc;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.logical.LogicalCalc;
+import org.apache.calcite.rel.logical.LogicalFilter;
+import org.apache.calcite.rel.logical.LogicalWindow;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexDynamicParam;
+import org.apache.calcite.rex.RexFieldAccess;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexLocalRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexOver;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexProgramBuilder;
+import org.apache.calcite.util.Util;
 
 import com.google.common.base.Function;
 import com.google.common.base.Predicate;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+
 /**
- * Rule that slices the {@link CalcRel} into sections which contain windowed
- * agg functions and sections which do not.
+ * Planner rule that slices a
+ * {@link org.apache.calcite.rel.core.Project}
+ * into sections which contain windowed
+ * aggregate functions and sections which do not.
  *
  * <p>The sections which contain windowed agg functions become instances of
- * {@link org.eigenbase.rel.WindowRel}. If the {@link CalcRel} does not contain any
- * windowed agg functions, does nothing.
+ * {@link org.apache.calcite.rel.logical.LogicalWindow}.
+ * If the {@link org.apache.calcite.rel.logical.LogicalCalc} does not contain
+ * any windowed agg functions, does nothing.
+ *
+ * <p>There is also a variant that matches
+ * {@link org.apache.calcite.rel.core.Calc} rather than {@code Project}.
  */
-public abstract class WindowedAggSplitterRule extends RelOptRule {
+public abstract class ProjectToWindowRule extends RelOptRule {
   //~ Static fields/initializers ---------------------------------------------
 
-  private static final Predicate<CalcRelBase> PREDICATE =
-      new Predicate<CalcRelBase>() {
-        public boolean apply(CalcRelBase calc) {
+  private static final Predicate<Calc> PREDICATE =
+      new Predicate<Calc>() {
+        public boolean apply(Calc calc) {
           return RexOver.containsOver(calc.getProgram());
         }
       };
 
-  private static final Predicate<ProjectRelBase> PREDICATE2 =
-      new Predicate<ProjectRelBase>() {
-        public boolean apply(ProjectRelBase project) {
+  private static final Predicate<Project> PREDICATE2 =
+      new Predicate<Project>() {
+        public boolean apply(Project project) {
           return RexOver.containsOver(project.getProjects(), null);
         }
       };
 
   /**
-   * Instance of the rule that applies to a {@link CalcRelBase} that contains
+   * Instance of the rule that applies to a
+   * {@link org.apache.calcite.rel.core.Calc} that contains
    * windowed aggregates and converts it into a mixture of
-   * {@link org.eigenbase.rel.WindowRel} and {@code CalcRelBase}.
+   * {@link org.apache.calcite.rel.logical.LogicalWindow} and {@code Calc}.
    */
-  public static final WindowedAggSplitterRule INSTANCE =
-      new WindowedAggSplitterRule(
-        operand(CalcRelBase.class, null, PREDICATE, any()),
-        "WindowedAggSplitterRule") {
+  public static final ProjectToWindowRule INSTANCE =
+      new ProjectToWindowRule(
+        operand(Calc.class, null, PREDICATE, any()),
+        "ProjectToWindowRule") {
         public void onMatch(RelOptRuleCall call) {
-          CalcRelBase calc = call.rel(0);
+          Calc calc = call.rel(0);
           assert RexOver.containsOver(calc.getProgram());
           CalcRelSplitter transform = new WindowedAggRelSplitter(calc);
           RelNode newRel = transform.execute();
@@ -74,18 +102,18 @@ public abstract class WindowedAggSplitterRule extends RelOptRule {
 
   /**
    * Instance of the rule that can be applied to a
-   * {@link org.eigenbase.rel.ProjectRelBase} and that produces, in turn,
-   * a mixture of {@code ProjectRel} and {@link org.eigenbase.rel.WindowRel}.
+   * {@link org.apache.calcite.rel.core.Project} and that produces, in turn,
+   * a mixture of {@code LogicalProject}
+   * and {@link org.apache.calcite.rel.logical.LogicalWindow}.
    */
-  public static final WindowedAggSplitterRule PROJECT =
-      new WindowedAggSplitterRule(
-        operand(ProjectRelBase.class, null, PREDICATE2, any()),
-        "WindowedAggSplitterRule:project") {
-        @Override
-        public void onMatch(RelOptRuleCall call) {
-          ProjectRelBase project = call.rel(0);
+  public static final ProjectToWindowRule PROJECT =
+      new ProjectToWindowRule(
+        operand(Project.class, null, PREDICATE2, any()),
+        "ProjectToWindowRule:project") {
+        @Override public void onMatch(RelOptRuleCall call) {
+          Project project = call.rel(0);
           assert RexOver.containsOver(project.getProjects(), null);
-          final RelNode child = project.getChild();
+          final RelNode child = project.getInput();
           final RelDataType rowType = project.getRowType();
           final RexProgram program =
               RexProgram.create(
@@ -94,9 +122,9 @@ public abstract class WindowedAggSplitterRule extends RelOptRule {
                   null,
                   project.getRowType(),
                   project.getCluster().getRexBuilder());
-          // temporary CalcRel, never registered
-          final CalcRel calc =
-              new CalcRel(
+          // temporary LogicalCalc, never registered
+          final LogicalCalc calc =
+              new LogicalCalc(
                   project.getCluster(),
                   project.getTraitSet(),
                   child,
@@ -104,14 +132,13 @@ public abstract class WindowedAggSplitterRule extends RelOptRule {
                   program,
                   ImmutableList.<RelCollation>of());
           CalcRelSplitter transform = new WindowedAggRelSplitter(calc) {
-            @Override
-            protected RelNode handle(RelNode rel) {
-              if (rel instanceof CalcRel) {
-                CalcRel calc = (CalcRel) rel;
+            @Override protected RelNode handle(RelNode rel) {
+              if (rel instanceof LogicalCalc) {
+                LogicalCalc calc = (LogicalCalc) rel;
                 final RexProgram program = calc.getProgram();
-                rel = calc.getChild();
+                rel = calc.getInput();
                 if (program.getCondition() != null) {
-                  rel = new FilterRel(
+                  rel = new LogicalFilter(
                       calc.getCluster(),
                       rel,
                       program.expandLocalRef(
@@ -140,11 +167,8 @@ public abstract class WindowedAggSplitterRule extends RelOptRule {
 
   //~ Constructors -----------------------------------------------------------
 
-  /**
-   * Creates a rule.
-   */
-  private WindowedAggSplitterRule(
-      RelOptRuleOperand operand, String description) {
+  /** Creates a ProjectToWindowRule. */
+  private ProjectToWindowRule(RelOptRuleOperand operand, String description) {
     super(operand, description);
   }
 
@@ -155,7 +179,7 @@ public abstract class WindowedAggSplitterRule extends RelOptRule {
    * (calls to {@link RexOver}) and ordinary expressions.
    */
   static class WindowedAggRelSplitter extends CalcRelSplitter {
-    WindowedAggRelSplitter(CalcRelBase calc) {
+    WindowedAggRelSplitter(Calc calc) {
       super(
           calc,
           new RelType[]{
@@ -223,15 +247,14 @@ public abstract class WindowedAggSplitterRule extends RelOptRule {
                 Util.permAssert(
                     program.getCondition() == null,
                     "WindowedAggregateRel cannot accept a condition");
-                return WindowRel.create(
+                return LogicalWindow.create(
                     cluster, traits, child, program, rowType);
               }
             }
           });
     }
 
-    @Override
-    protected List<Set<Integer>> getCohorts() {
+    @Override protected List<Set<Integer>> getCohorts() {
       // Here used to be the implementation that treats all the RexOvers
       // as a single Cohort. This is flawed if the RexOvers
       // depend on each other (i.e. the second one uses the result
@@ -241,4 +264,4 @@ public abstract class WindowedAggSplitterRule extends RelOptRule {
   }
 }
 
-// End WindowedAggSplitterRule.java
+// End ProjectToWindowRule.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/rules/PushProjector.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/PushProjector.java b/core/src/main/java/org/apache/calcite/rel/rules/PushProjector.java
index 4709439..d317237 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/PushProjector.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/PushProjector.java
@@ -14,24 +14,33 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
-
-import java.util.*;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.rex.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.util.Pair;
-
-import net.hydromatic.linq4j.Ord;
-
-import net.hydromatic.optiq.util.BitSets;
+package org.apache.calcite.rel.rules;
+
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.SetOp;
+import org.apache.calcite.rel.logical.LogicalProject;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.Pair;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.List;
+import java.util.Set;
+
 /**
  * PushProjector is a utility class used to perform operations used in push
  * projection rules.
@@ -49,7 +58,7 @@ import com.google.common.collect.Lists;
 public class PushProjector {
   //~ Instance fields --------------------------------------------------------
 
-  private final ProjectRel origProj;
+  private final LogicalProject origProj;
   private final RexNode origFilter;
   private final RelNode childRel;
   private final ExprCondition preserveExprCondition;
@@ -177,7 +186,7 @@ public class PushProjector {
    *                              be preserved in the projection
    */
   public PushProjector(
-      ProjectRel origProj,
+      LogicalProject origProj,
       RexNode origFilter,
       RelNode childRel,
       ExprCondition preserveExprCondition) {
@@ -196,8 +205,8 @@ public class PushProjector {
     nChildFields = childFields.size();
 
     projRefs = new BitSet(nChildFields);
-    if (childRel instanceof JoinRelBase) {
-      JoinRelBase joinRel = (JoinRelBase) childRel;
+    if (childRel instanceof Join) {
+      Join joinRel = (Join) childRel;
       List<RelDataTypeField> leftFields =
           joinRel.getLeft().getRowType().getFieldList();
       List<RelDataTypeField> rightFields =
@@ -358,8 +367,8 @@ public class PushProjector {
     assert nSystemProject + nProject + nRightProject
         == projRefs.cardinality();
 
-    if ((childRel instanceof JoinRelBase)
-        || (childRel instanceof SetOpRel)) {
+    if ((childRel instanceof Join)
+        || (childRel instanceof SetOp)) {
       // if nothing is projected from the children, arbitrarily project
       // the first columns; this is necessary since Fennel doesn't
       // handle 0-column projections
@@ -367,7 +376,7 @@ public class PushProjector {
         projRefs.set(0);
         nProject = 1;
       }
-      if (childRel instanceof JoinRelBase) {
+      if (childRel instanceof Join) {
         if ((nRightProject == 0) && (rightPreserveExprs.size() == 0)) {
           projRefs.set(nFields);
           nRightProject = 1;
@@ -400,7 +409,7 @@ public class PushProjector {
    *                  of a join
    * @return created projection
    */
-  public ProjectRel createProjectRefsAndExprs(
+  public LogicalProject createProjectRefsAndExprs(
       RelNode projChild,
       boolean adjust,
       boolean rightSide) {
@@ -463,7 +472,7 @@ public class PushProjector {
               ((RexCall) projExpr).getOperator().getName()));
     }
 
-    return (ProjectRel) RelOptUtil.createProject(
+    return (LogicalProject) RelOptUtil.createProject(
         projChild,
         Pair.left(newProjects),
         Pair.right(newProjects));
@@ -737,7 +746,7 @@ public class PushProjector {
   /**
    * A functor that replies true or false for a given expression.
    *
-   * @see org.eigenbase.rel.rules.PushProjector.OperatorExprCondition
+   * @see org.apache.calcite.rel.rules.PushProjector.OperatorExprCondition
    */
   public interface ExprCondition {
     /**

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 d89561d..9d9cc2d 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
@@ -14,38 +14,55 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
-
-import java.math.*;
-
-import java.util.*;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.rex.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.fun.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.util.*;
-
-import net.hydromatic.linq4j.Ord;
+package org.apache.calcite.rel.rules;
+
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.logical.LogicalCalc;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexProgramBuilder;
+import org.apache.calcite.rex.RexShuttle;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlOperator;
+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.Pair;
+import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
 
-import static org.eigenbase.util.Static.RESOURCE;
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.calcite.util.Static.RESOURCE;
 
 /**
  * ReduceDecimalsRule is a rule which reduces decimal operations (such as casts
  * or arithmetic) into operations involving more primitive types (such as longs
- * and doubles). The rule allows eigenbase implementations to deal with decimals
+ * and doubles). The rule allows Calcite implementations to deal with decimals
  * in a consistent manner, while saving the effort of implementing them.
  *
- * <p>The rule can be applied to a {@link CalcRel} with a program for which
+ * <p>The rule can be applied to a
+ * {@link org.apache.calcite.rel.logical.LogicalCalc} with a program for which
  * {@link RexUtil#requiresDecimalExpansion} returns true. The rule relies on a
  * {@link RexShuttle} to walk over relational expressions and replace them.
  *
- * <p>While decimals are generally not implemented by the eigenbase runtime, the
+ * <p>While decimals are generally not implemented by the Calcite runtime, the
  * rule is optionally applied, in order to support the situation in which we
  * would like to push down decimal operations to an external database.
  */
@@ -58,7 +75,7 @@ public class ReduceDecimalsRule extends RelOptRule {
    * Creates a ReduceDecimalsRule.
    */
   private ReduceDecimalsRule() {
-    super(operand(CalcRel.class, any()));
+    super(operand(LogicalCalc.class, any()));
   }
 
   //~ Methods ----------------------------------------------------------------
@@ -70,21 +87,21 @@ public class ReduceDecimalsRule extends RelOptRule {
 
   // implement RelOptRule
   public void onMatch(RelOptRuleCall call) {
-    CalcRel calcRel = call.rel(0);
+    LogicalCalc calc = call.rel(0);
 
     // Expand decimals in every expression in this program. If no
     // expression changes, don't apply the rule.
-    final RexProgram program = calcRel.getProgram();
+    final RexProgram program = calc.getProgram();
     if (!RexUtil.requiresDecimalExpansion(program, true)) {
       return;
     }
 
-    final RexBuilder rexBuilder = calcRel.getCluster().getRexBuilder();
+    final RexBuilder rexBuilder = calc.getCluster().getRexBuilder();
     final RexShuttle shuttle = new DecimalShuttle(rexBuilder);
     RexProgramBuilder programBuilder =
         RexProgramBuilder.create(
             rexBuilder,
-            calcRel.getChild().getRowType(),
+            calc.getInput().getRowType(),
             program.getExprList(),
             program.getProjectList(),
             program.getCondition(),
@@ -93,15 +110,15 @@ public class ReduceDecimalsRule extends RelOptRule {
             true);
 
     final RexProgram newProgram = programBuilder.getProgram();
-    CalcRel newCalcRel =
-        new CalcRel(
-            calcRel.getCluster(),
-            calcRel.getTraitSet(),
-            calcRel.getChild(),
+    LogicalCalc newCalc =
+        new LogicalCalc(
+            calc.getCluster(),
+            calc.getTraitSet(),
+            calc.getInput(),
             newProgram.getOutputRowType(),
             newProgram,
             Collections.<RelCollation>emptyList());
-    call.transformTo(newCalcRel);
+    call.transformTo(newCalc);
   }
 
   //~ Inner Classes ----------------------------------------------------------
@@ -267,10 +284,10 @@ public class ReduceDecimalsRule extends RelOptRule {
    * Rewrites a decimal expression for a specific set of SqlOperator's. In
    * general, most expressions are rewritten in such a way that SqlOperator's
    * do not have to deal with decimals. Decimals are represented by their
-   * unscaled integer representations, similar to {@link
-   * BigDecimal#unscaledValue()} (i.e. 10^scale). Once decimals are decoded,
-   * SqlOperators can then operate on the integer representations. The value
-   * can later be recoded as a decimal.
+   * unscaled integer representations, similar to
+   * {@link BigDecimal#unscaledValue()} (i.e. 10^scale). Once decimals are
+   * decoded, SqlOperators can then operate on the integer representations. The
+   * value can later be recoded as a decimal.
    *
    * <p>For example, suppose one casts 2.0 as a decima(10,4). The value is
    * decoded (20), multiplied by a scale factor (1000), for a result of
@@ -307,12 +324,11 @@ public class ReduceDecimalsRule extends RelOptRule {
     }
 
     /**
-     * This defaults to the utility method, {@link
-     * RexUtil#requiresDecimalExpansion(RexNode, boolean)} which checks
-     * general guidelines on whether a rewrite should be considered at all.
-     * In general, it is helpful to update the utility method since that
-     * method is often used to filter the somewhat expensive rewrite
-     * process.
+     * This defaults to the utility method,
+     * {@link RexUtil#requiresDecimalExpansion(RexNode, boolean)} which checks
+     * general guidelines on whether a rewrite should be considered at all.  In
+     * general, it is helpful to update the utility method since that method is
+     * often used to filter the somewhat expensive rewrite process.
      *
      * <p>However, this method provides another place for implementations of
      * RexExpander to make a more detailed analysis before deciding on
@@ -420,9 +436,9 @@ public class ReduceDecimalsRule extends RelOptRule {
 
     /**
      * Scales down a decimal value, and returns the scaled value as an exact
-     * numeric. with the rounding convention {@link BigDecimal#ROUND_HALF_UP
-     * BigDecimal.ROUND_HALF_UP}. (Values midway between two points are
-     * rounded away from zero.)
+     * numeric. with the rounding convention
+     * {@link BigDecimal#ROUND_HALF_UP BigDecimal.ROUND_HALF_UP}. (Values midway
+     * between two points are rounded away from zero.)
      *
      * @param value the integer representation of a decimal
      * @param scale a value from zero to max precision
@@ -522,8 +538,7 @@ public class ReduceDecimalsRule extends RelOptRule {
 
       // TODO: make a validator exception for this
       if (scaleDiff >= maxPrecision) {
-        throw Util.needToImplement(
-            "Source type with scale " + scale
+        throw Util.needToImplement("Source type with scale " + scale
             + " cannot be converted to target type with scale "
             + required + " because the smallest value of the "
             + "source type is too large to be encoded by the "
@@ -762,7 +777,7 @@ public class ReduceDecimalsRule extends RelOptRule {
           || !SqlTypeUtil.isExactNumeric(toType)) {
         throw Util.needToImplement(
             "Cast from '" + fromType.toString()
-            + "' to '" + toType.toString() + "'");
+                + "' to '" + toType.toString() + "'");
       }
       int fromScale = fromType.getScale();
       int toScale = toType.getScale();

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/rules/ReduceExpressionsRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/ReduceExpressionsRule.java b/core/src/main/java/org/apache/calcite/rel/rules/ReduceExpressionsRule.java
index 7025a47..5fb7035 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/ReduceExpressionsRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/ReduceExpressionsRule.java
@@ -14,19 +14,46 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
-
-import java.util.*;
-import java.util.regex.*;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.rex.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.fun.*;
-import org.eigenbase.util.Stacks;
-import org.eigenbase.util.Util;
+package org.apache.calcite.rel.rules;
+
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Empty;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.logical.LogicalCalc;
+import org.apache.calcite.rel.logical.LogicalFilter;
+import org.apache.calcite.rel.logical.LogicalProject;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexCorrelVariable;
+import org.apache.calcite.rex.RexDynamicParam;
+import org.apache.calcite.rex.RexFieldAccess;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexLocalRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexOver;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexProgramBuilder;
+import org.apache.calcite.rex.RexRangeRef;
+import org.apache.calcite.rex.RexShuttle;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.fun.SqlRowOperator;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.util.Stacks;
+import org.apache.calcite.util.Util;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.regex.Pattern;
 
 /**
  * Collection of planner rules that apply various simplifying transformations on
@@ -44,22 +71,23 @@ public abstract class ReduceExpressionsRule extends RelOptRule {
 
   /**
    * Regular expression that matches the description of all instances of this
-   * rule and {@link ReduceValuesRule} also. Use
+   * rule and {@link ValuesReduceRule} also. Use
    * it to prevent the planner from invoking these rules.
    */
   public static final Pattern EXCLUSION_PATTERN =
       Pattern.compile("Reduce(Expressions|Values)Rule.*");
 
   /**
-   * Singleton rule that reduces constants inside a {@link FilterRel}. If the
-   * condition is a constant, the filter is removed (if TRUE) or replaced with
-   * {@link EmptyRel} (if FALSE or NULL).
+   * Singleton rule that reduces constants inside a
+   * {@link org.apache.calcite.rel.logical.LogicalFilter}. If the condition is a
+   * constant, the filter is removed (if TRUE) or replaced with
+   * {@link org.apache.calcite.rel.core.Empty} (if FALSE or NULL).
    */
   public static final ReduceExpressionsRule FILTER_INSTANCE =
-      new ReduceExpressionsRule(FilterRel.class,
+      new ReduceExpressionsRule(LogicalFilter.class,
           "ReduceExpressionsRule[Filter]") {
         public void onMatch(RelOptRuleCall call) {
-          FilterRel filter = call.rel(0);
+          LogicalFilter filter = call.rel(0);
           List<RexNode> expList = new ArrayList<RexNode>(filter.getChildExps());
           RexNode newConditionExp;
           boolean reduced;
@@ -77,18 +105,18 @@ public abstract class ReduceExpressionsRule extends RelOptRule {
           }
           if (newConditionExp.isAlwaysTrue()) {
             call.transformTo(
-                filter.getChild());
+                filter.getInput());
           } else if (
               (newConditionExp instanceof RexLiteral)
                   || RexUtil.isNullLiteral(newConditionExp, true)) {
             call.transformTo(
-                new EmptyRel(
+                new Empty(
                     filter.getCluster(),
                     filter.getRowType()));
           } else if (reduced) {
             call.transformTo(
                 RelOptUtil.createFilter(
-                    filter.getChild(),
+                    filter.getInput(),
                     expList.get(0)));
           } else {
             if (newConditionExp instanceof RexCall) {
@@ -110,12 +138,12 @@ public abstract class ReduceExpressionsRule extends RelOptRule {
 
         private void reduceNotNullableFilter(
             RelOptRuleCall call,
-            FilterRel filter,
+            LogicalFilter filter,
             RexCall rexCall,
             boolean reverse) {
           // If the expression is a IS [NOT] NULL on a non-nullable
           // column, then we can either remove the filter or replace
-          // it with an EmptyRel.
+          // it with an Empty.
           SqlOperator op = rexCall.getOperator();
           boolean alwaysTrue;
           switch (rexCall.getKind()) {
@@ -137,10 +165,10 @@ public abstract class ReduceExpressionsRule extends RelOptRule {
             RexInputRef inputRef = (RexInputRef) operand;
             if (!inputRef.getType().isNullable()) {
               if (alwaysTrue) {
-                call.transformTo(filter.getChild());
+                call.transformTo(filter.getInput());
               } else {
                 call.transformTo(
-                    new EmptyRel(
+                    new Empty(
                         filter.getCluster(),
                         filter.getRowType()));
               }
@@ -150,21 +178,21 @@ public abstract class ReduceExpressionsRule extends RelOptRule {
       };
 
   public static final ReduceExpressionsRule PROJECT_INSTANCE =
-      new ReduceExpressionsRule(ProjectRel.class,
+      new ReduceExpressionsRule(LogicalProject.class,
           "ReduceExpressionsRule[Project]") {
         public void onMatch(RelOptRuleCall call) {
-          ProjectRel project = call.rel(0);
+          LogicalProject project = call.rel(0);
           List<RexNode> expList =
               new ArrayList<RexNode>(project.getChildExps());
           if (reduceExpressions(project, expList)) {
             call.transformTo(
-                new ProjectRel(
+                new LogicalProject(
                     project.getCluster(),
                     project.getTraitSet(),
-                    project.getChild(),
+                    project.getInput(),
                     expList,
                     project.getRowType(),
-                    ProjectRel.Flags.BOXED));
+                    LogicalProject.Flags.BOXED));
 
             // New plan is absolutely better than old plan.
             call.getPlanner().setImportance(project, 0.0);
@@ -173,10 +201,10 @@ public abstract class ReduceExpressionsRule extends RelOptRule {
       };
 
   public static final ReduceExpressionsRule JOIN_INSTANCE =
-      new ReduceExpressionsRule(JoinRelBase.class,
+      new ReduceExpressionsRule(Join.class,
           "ReduceExpressionsRule[Join]") {
         public void onMatch(RelOptRuleCall call) {
-          final JoinRelBase join = call.rel(0);
+          final Join join = call.rel(0);
           List<RexNode> expList = new ArrayList<RexNode>(join.getChildExps());
           if (reduceExpressions(join, expList)) {
             call.transformTo(
@@ -195,9 +223,10 @@ public abstract class ReduceExpressionsRule extends RelOptRule {
       };
 
   public static final ReduceExpressionsRule CALC_INSTANCE =
-      new ReduceExpressionsRule(CalcRel.class, "ReduceExpressionsRule[Calc]") {
+      new ReduceExpressionsRule(LogicalCalc.class,
+          "ReduceExpressionsRule[Calc]") {
         public void onMatch(RelOptRuleCall call) {
-          CalcRel calc = call.rel(0);
+          LogicalCalc calc = call.rel(0);
           RexProgram program = calc.getProgram();
           final List<RexNode> exprList = program.getExprList();
 
@@ -216,7 +245,7 @@ public abstract class ReduceExpressionsRule extends RelOptRule {
           if (reduceExpressions(calc, expandedExprList)) {
             final RexProgramBuilder builder =
                 new RexProgramBuilder(
-                    calc.getChild().getRowType(),
+                    calc.getInput().getRowType(),
                     calc.getCluster().getRexBuilder());
             List<RexLocalRef> list = new ArrayList<RexLocalRef>();
             for (RexNode expr : expandedExprList) {
@@ -234,7 +263,7 @@ public abstract class ReduceExpressionsRule extends RelOptRule {
                 // condition is always NULL or FALSE - replace calc
                 // with empty
                 call.transformTo(
-                    new EmptyRel(
+                    new Empty(
                         calc.getCluster(),
                         calc.getRowType()));
                 return;
@@ -250,10 +279,10 @@ public abstract class ReduceExpressionsRule extends RelOptRule {
                   program.getOutputRowType().getFieldNames().get(k++));
             }
             call.transformTo(
-                new CalcRel(
+                new LogicalCalc(
                     calc.getCluster(),
                     calc.getTraitSet(),
-                    calc.getChild(),
+                    calc.getInput(),
                     calc.getRowType(),
                     builder.getProgram(),
                     calc.getCollationList()));
@@ -328,14 +357,14 @@ public abstract class ReduceExpressionsRule extends RelOptRule {
     List<RexNode> reducedValues = new ArrayList<RexNode>();
     executor.reduce(rexBuilder, constExps, reducedValues);
 
-    // For ProjectRel, we have to be sure to preserve the result
+    // For Project, we have to be sure to preserve the result
     // types, so always cast regardless of the expression type.
-    // For other RelNodes like FilterRel, in general, this isn't necessary,
+    // For other RelNodes like Filter, in general, this isn't necessary,
     // and the presence of casts could hinder other rules such as sarg
     // analysis, which require bare literals.  But there are special cases,
     // like when the expression is a UDR argument, that need to be
     // handled as special cases.
-    if (rel instanceof ProjectRel) {
+    if (rel instanceof LogicalProject) {
       for (int i = 0; i < reducedValues.size(); i++) {
         addCasts.set(i, true);
       }
@@ -403,8 +432,7 @@ public abstract class ReduceExpressionsRule extends RelOptRule {
       this.addCasts = addCasts;
     }
 
-    @Override
-    public RexNode visitCall(final RexCall call) {
+    @Override public RexNode visitCall(final RexCall call) {
       int i = reducibleExps.indexOf(call);
       if (i == -1) {
         return super.visitCall(call);
@@ -418,7 +446,7 @@ public abstract class ReduceExpressionsRule extends RelOptRule {
         //
         // Also, we cannot reduce CAST('abc' AS VARCHAR(4)) to 'abc'.
         // If we make 'abc' of type VARCHAR(4), we may later encounter
-        // the same expression in a ProjectRel's digest where it has
+        // the same expression in a Project's digest where it has
         // type VARCHAR(3), and that's wrong.
         replacement =
             rexBuilder.makeCast(

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/rules/SemiJoinFilterTransposeRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/SemiJoinFilterTransposeRule.java b/core/src/main/java/org/apache/calcite/rel/rules/SemiJoinFilterTransposeRule.java
index 4e961db..9fa288d 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/SemiJoinFilterTransposeRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/SemiJoinFilterTransposeRule.java
@@ -14,49 +14,55 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
+package org.apache.calcite.rel.rules;
 
-import org.eigenbase.rel.*;
-import org.eigenbase.rel.RelFactories.FilterFactory;
-import org.eigenbase.relopt.*;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.SemiJoin;
+import org.apache.calcite.rel.logical.LogicalFilter;
 
 /**
- * PushSemiJoinPastFilterRule implements the rule for pushing semijoins down in
- * a tree past a filter in order to trigger other rules that will convert
- * semijoins. SemiJoinRel(FilterRel(X), Y) &rarr; FilterRel(SemiJoinRel(X, Y))
+ * Planner rule that pushes
+ * {@link org.apache.calcite.rel.core.SemiJoin}s down in a tree past
+ * a {@link org.apache.calcite.rel.core.Filter}.
+ *
+ * <p>The intention is to trigger other rules that will convert
+ * {@code SemiJoin}s.
+ *
+ * <p>SemiJoin(LogicalFilter(X), Y) &rarr; LogicalFilter(SemiJoin(X, Y))
+ *
+ * @see SemiJoinProjectTransposeRule
  */
-public class PushSemiJoinPastFilterRule extends RelOptRule {
-  public static final PushSemiJoinPastFilterRule INSTANCE =
-      new PushSemiJoinPastFilterRule(RelFactories.DEFAULT_FILTER_FACTORY);
-
-  private final FilterFactory filterFactory;
+public class SemiJoinFilterTransposeRule extends RelOptRule {
+  public static final SemiJoinFilterTransposeRule INSTANCE =
+      new SemiJoinFilterTransposeRule();
 
   //~ Constructors -----------------------------------------------------------
 
   /**
-   * Creates a PushSemiJoinPastFilterRule.
-   * @param filterFactory Factory to create Filter
+   * Creates a SemiJoinFilterTransposeRule.
    */
-  public PushSemiJoinPastFilterRule(FilterFactory filterFactory) {
+  private SemiJoinFilterTransposeRule() {
     super(
-        operand(
-            SemiJoinRel.class,
-            some(operand(FilterRelBase.class, any()))));
-    this.filterFactory = filterFactory;
+        operand(SemiJoin.class,
+            some(operand(LogicalFilter.class, any()))));
   }
 
   //~ Methods ----------------------------------------------------------------
 
   // implement RelOptRule
   public void onMatch(RelOptRuleCall call) {
-    SemiJoinRel semiJoin = call.rel(0);
-    FilterRelBase filter = call.rel(1);
+    SemiJoin semiJoin = call.rel(0);
+    LogicalFilter filter = call.rel(1);
 
     RelNode newSemiJoin =
-        new SemiJoinRel(
+        new SemiJoin(
             semiJoin.getCluster(),
             semiJoin.getCluster().traitSetOf(Convention.NONE),
-            filter.getChild(),
+            filter.getInput(),
             semiJoin.getRight(),
             semiJoin.getCondition(),
             semiJoin.getLeftKeys(),
@@ -65,10 +71,10 @@ public class PushSemiJoinPastFilterRule extends RelOptRule {
     RelNode newFilter =
         RelOptUtil.createFilter(
             newSemiJoin,
-            filter.getCondition(), filterFactory);
+            filter.getCondition());
 
     call.transformTo(newFilter);
   }
 }
 
-// End PushSemiJoinPastFilterRule.java
+// End SemiJoinFilterTransposeRule.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/rules/SemiJoinJoinTransposeRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/SemiJoinJoinTransposeRule.java b/core/src/main/java/org/apache/calcite/rel/rules/SemiJoinJoinTransposeRule.java
index 21cde7f..a859161 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/SemiJoinJoinTransposeRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/SemiJoinJoinTransposeRule.java
@@ -14,53 +14,58 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
-
-import java.util.*;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.rex.*;
-import org.eigenbase.util.ImmutableIntList;
+package org.apache.calcite.rel.rules;
+
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.SemiJoin;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.util.ImmutableIntList;
+
+import java.util.ArrayList;
+import java.util.List;
 
 /**
- * PushSemiJoinPastJoinRule implements the rule for pushing semi-joins down in a
- * tree past a join in order to trigger other rules that will convert
- * semi-joins.
+ * Planner rule that pushes a {@link org.apache.calcite.rel.core.SemiJoin}
+ * down in a tree past a {@link org.apache.calcite.rel.core.Join}
+ * in order to trigger other rules that will convert {@code SemiJoin}s.
  *
  * <ul>
- * <li>SemiJoinRel(JoinRel(X, Y), Z) &rarr; JoinRel(SemiJoinRel(X, Z), Y)
- * <li>SemiJoinRel(JoinRel(X, Y), Z) &rarr; JoinRel(X, SemiJoinRel(Y, Z))
+ * <li>SemiJoin(LogicalJoin(X, Y), Z) &rarr; LogicalJoin(SemiJoin(X, Z), Y)
+ * <li>SemiJoin(LogicalJoin(X, Y), Z) &rarr; LogicalJoin(X, SemiJoin(Y, Z))
  * </ul>
  *
  * <p>Whether this
  * first or second conversion is applied depends on which operands actually
  * participate in the semi-join.</p>
  */
-public class PushSemiJoinPastJoinRule extends RelOptRule {
-  public static final PushSemiJoinPastJoinRule INSTANCE =
-      new PushSemiJoinPastJoinRule();
+public class SemiJoinJoinTransposeRule extends RelOptRule {
+  public static final SemiJoinJoinTransposeRule INSTANCE =
+      new SemiJoinJoinTransposeRule();
 
   //~ Constructors -----------------------------------------------------------
 
   /**
-   * Creates a PushSemiJoinPastJoinRule.
+   * Creates a SemiJoinJoinTransposeRule.
    */
-  private PushSemiJoinPastJoinRule() {
+  private SemiJoinJoinTransposeRule() {
     super(
-        operand(
-            SemiJoinRel.class,
-            some(operand(JoinRelBase.class, any()))));
+        operand(SemiJoin.class,
+            some(operand(Join.class, any()))));
   }
 
   //~ Methods ----------------------------------------------------------------
 
   // implement RelOptRule
   public void onMatch(RelOptRuleCall call) {
-    SemiJoinRel semiJoin = call.rel(0);
-    final JoinRelBase join = call.rel(1);
-    if (join instanceof SemiJoinRel) {
+    SemiJoin semiJoin = call.rel(0);
+    final Join join = call.rel(1);
+    if (join instanceof SemiJoin) {
       return;
     }
     final ImmutableIntList leftKeys = semiJoin.getLeftKeys();
@@ -150,8 +155,8 @@ public class PushSemiJoinPastJoinRule extends RelOptRule {
     } else {
       leftSemiJoinOp = join.getRight();
     }
-    SemiJoinRel newSemiJoin =
-        new SemiJoinRel(
+    SemiJoin newSemiJoin =
+        new SemiJoin(
             semiJoin.getCluster(),
             semiJoin.getCluster().traitSetOf(Convention.NONE),
             leftSemiJoinOp,
@@ -216,4 +221,4 @@ public class PushSemiJoinPastJoinRule extends RelOptRule {
   }
 }
 
-// End PushSemiJoinPastJoinRule.java
+// End SemiJoinJoinTransposeRule.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/rules/SemiJoinProjectTransposeRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/SemiJoinProjectTransposeRule.java b/core/src/main/java/org/apache/calcite/rel/rules/SemiJoinProjectTransposeRule.java
index 9d7d4b0..dfae5f0 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/SemiJoinProjectTransposeRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/SemiJoinProjectTransposeRule.java
@@ -14,52 +14,63 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
-
-import java.util.*;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.rel.RelFactories.ProjectFactory;
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.rex.*;
-import org.eigenbase.util.ImmutableIntList;
-import org.eigenbase.util.Pair;
+package org.apache.calcite.rel.rules;
+
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.core.SemiJoin;
+import org.apache.calcite.rel.logical.LogicalProject;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexProgramBuilder;
+import org.apache.calcite.util.ImmutableIntList;
+import org.apache.calcite.util.Pair;
+
+import java.util.ArrayList;
+import java.util.List;
 
 /**
- * PushSemiJoinPastProjectRule implements the rule for pushing semijoins down in
- * a tree past a project in order to trigger other rules that will convert
- * semijoins.
+ * Planner rule that pushes
+ * a {@link org.apache.calcite.rel.core.SemiJoin} down in a tree past
+ * a {@link org.apache.calcite.rel.core.Project}.
+ *
+ * <p>The intention is to trigger other rules that will convert
+ * {@code SemiJoin}s.
  *
- * <p>SemiJoinRel(ProjectRel(X), Y) &rarr; ProjectRel(SemiJoinRel(X, Y))
+ * <p>SemiJoin(LogicalProject(X), Y) &rarr; LogicalProject(SemiJoin(X, Y))
+ *
+ * @see org.apache.calcite.rel.rules.SemiJoinFilterTransposeRule
  */
-public class PushSemiJoinPastProjectRule extends RelOptRule {
-  public static final PushSemiJoinPastProjectRule INSTANCE =
-      new PushSemiJoinPastProjectRule(RelFactories.DEFAULT_PROJECT_FACTORY);
-
-  private final RelFactories.ProjectFactory projectFactory;
+public class SemiJoinProjectTransposeRule extends RelOptRule {
+  public static final SemiJoinProjectTransposeRule INSTANCE =
+      new SemiJoinProjectTransposeRule();
 
   //~ Constructors -----------------------------------------------------------
 
   /**
-   * Creates a PushSemiJoinPastProjectRule.
-   *
-   * @param projectFactory factory to create Project
+   * Creates a SemiJoinProjectTransposeRule.
    */
-  public PushSemiJoinPastProjectRule(ProjectFactory projectFactory) {
+  private SemiJoinProjectTransposeRule() {
     super(
-        operand(
-            SemiJoinRel.class,
-            some(operand(ProjectRel.class, any()))));
-    this.projectFactory = projectFactory;
+        operand(SemiJoin.class,
+            some(operand(LogicalProject.class, any()))));
   }
 
   //~ Methods ----------------------------------------------------------------
 
-  // implement RelOptRule
   public void onMatch(RelOptRuleCall call) {
-    SemiJoinRel semiJoin = call.rel(0);
-    ProjectRel project = call.rel(1);
+    SemiJoin semiJoin = call.rel(0);
+    LogicalProject project = call.rel(1);
 
     // convert the LHS semijoin keys to reference the child projection
     // expression; all projection expressions must be RexInputRefs,
@@ -76,11 +87,11 @@ public class PushSemiJoinPastProjectRule extends RelOptRule {
     // pulled up
     RexNode newCondition = adjustCondition(project, semiJoin);
 
-    SemiJoinRel newSemiJoin =
-        new SemiJoinRel(
+    SemiJoin newSemiJoin =
+        new SemiJoin(
             semiJoin.getCluster(),
             semiJoin.getCluster().traitSetOf(Convention.NONE),
-            project.getChild(),
+            project.getInput(),
             semiJoin.getRight(),
             newCondition,
             ImmutableIntList.copyOf(newLeftKeys),
@@ -90,7 +101,7 @@ public class PushSemiJoinPastProjectRule extends RelOptRule {
     // are the same as the original because they only reference the LHS
     // of the semijoin and the semijoin only projects out the LHS
     RelNode newProject =
-        projectFactory.createProject(
+        RelOptUtil.createProject(
             newSemiJoin,
             projExprs,
             project.getRowType().getFieldNames());
@@ -104,11 +115,11 @@ public class PushSemiJoinPastProjectRule extends RelOptRule {
    * that references to the LHS of a semijoin should now reference the
    * children of the project that's on the LHS.
    *
-   * @param project  ProjectRel on the LHS of the semijoin
+   * @param project  LogicalProject on the LHS of the semijoin
    * @param semiJoin the semijoin
    * @return the modified semijoin condition
    */
-  private RexNode adjustCondition(ProjectRel project, SemiJoinRel semiJoin) {
+  private RexNode adjustCondition(LogicalProject project, SemiJoin semiJoin) {
     // create two RexPrograms -- the bottom one representing a
     // concatenation of the project and the RHS of the semijoin and the
     // top one representing the semijoin condition
@@ -120,8 +131,8 @@ public class PushSemiJoinPastProjectRule extends RelOptRule {
     // for the bottom RexProgram, the input is a concatenation of the
     // child of the project and the RHS of the semijoin
     RelDataType bottomInputRowType =
-        JoinRelBase.deriveJoinRowType(
-            project.getChild().getRowType(),
+        Join.deriveJoinRowType(
+            project.getInput().getRowType(),
             rightChild.getRowType(),
             JoinRelType.INNER,
             typeFactory,
@@ -135,7 +146,7 @@ public class PushSemiJoinPastProjectRule extends RelOptRule {
     for (Pair<RexNode, String> pair : project.getNamedProjects()) {
       bottomProgramBuilder.addProject(pair.left, pair.right);
     }
-    int nLeftFields = project.getChild().getRowType().getFieldCount();
+    int nLeftFields = project.getInput().getRowType().getFieldCount();
     List<RelDataTypeField> rightFields =
         rightChild.getRowType().getFieldList();
     int nRightFields = rightFields.size();
@@ -151,7 +162,7 @@ public class PushSemiJoinPastProjectRule extends RelOptRule {
     // input rowtype into the top program is the concatenation of the
     // project and the RHS of the semijoin
     RelDataType topInputRowType =
-        JoinRelBase.deriveJoinRowType(
+        Join.deriveJoinRowType(
             project.getRowType(),
             rightChild.getRowType(),
             JoinRelType.INNER,
@@ -180,4 +191,4 @@ public class PushSemiJoinPastProjectRule extends RelOptRule {
   }
 }
 
-// End PushSemiJoinPastProjectRule.java
+// End SemiJoinProjectTransposeRule.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/rules/SemiJoinRemoveRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/SemiJoinRemoveRule.java b/core/src/main/java/org/apache/calcite/rel/rules/SemiJoinRemoveRule.java
index 40f2bd7..4c6a20d 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/SemiJoinRemoveRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/SemiJoinRemoveRule.java
@@ -14,36 +14,39 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
+package org.apache.calcite.rel.rules;
 
-import org.eigenbase.relopt.*;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.rel.core.SemiJoin;
 
 /**
- * RemoveSemiJoinRule implements the rule that removes semijoins from a join
- * tree if it turns out it's not possible to convert a SemiJoinRel to an indexed
- * scan on a join factor. Namely, if the join factor does not reduce to a single
- * table that can be scanned using an index. This rule should only be applied
- * after attempts have been made to convert SemiJoinRels.
+ * Planner rule that removes a {@link org.apache.calcite.rel.core.SemiJoin}s
+ * from a join tree.
+ *
+ * <p>It is invoked after attempts have been made to convert a SemiJoin to an
+ * indexed scan on a join factor have failed. Namely, if the join factor does
+ * not reduce to a single table that can be scanned using an index.
+ *
+ * <p>It should only be enabled if all SemiJoins in the plan are advisory; that
+ * is, they can be safely dropped without affecting the semantics of the query.
  */
-public class RemoveSemiJoinRule extends RelOptRule {
-  public static final RemoveSemiJoinRule INSTANCE =
-      new RemoveSemiJoinRule();
+public class SemiJoinRemoveRule extends RelOptRule {
+  public static final SemiJoinRemoveRule INSTANCE =
+      new SemiJoinRemoveRule();
 
   //~ Constructors -----------------------------------------------------------
 
-  /**
-   * Creates a RemoveSemiJoinRule.
-   */
-  private RemoveSemiJoinRule() {
-    super(operand(SemiJoinRel.class, any()));
+  /** Creates a SemiJoinRemoveRule. */
+  private SemiJoinRemoveRule() {
+    super(operand(SemiJoin.class, any()));
   }
 
   //~ Methods ----------------------------------------------------------------
 
-  // implement RelOptRule
   public void onMatch(RelOptRuleCall call) {
     call.transformTo(call.rel(0).getInput(0));
   }
 }
 
-// End RemoveSemiJoinRule.java
+// End SemiJoinRemoveRule.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/rules/SemiJoinRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/SemiJoinRule.java b/core/src/main/java/org/apache/calcite/rel/rules/SemiJoinRule.java
index cd06dd8..bedea88 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/SemiJoinRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/SemiJoinRule.java
@@ -14,49 +14,48 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
+package org.apache.calcite.rel.rules;
 
-import java.util.BitSet;
-import java.util.List;
-
-import org.eigenbase.rel.AggregateRelBase;
-import org.eigenbase.rel.JoinInfo;
-import org.eigenbase.rel.JoinRelBase;
-import org.eigenbase.rel.ProjectRelBase;
-import org.eigenbase.rel.RelNode;
-import org.eigenbase.relopt.Convention;
-import org.eigenbase.relopt.RelOptRule;
-import org.eigenbase.relopt.RelOptRuleCall;
-import org.eigenbase.relopt.RelOptUtil;
-import org.eigenbase.util.ImmutableIntList;
-import org.eigenbase.util.IntList;
-
-import net.hydromatic.optiq.util.BitSets;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Aggregate;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.JoinInfo;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.SemiJoin;
+import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.ImmutableIntList;
+import org.apache.calcite.util.IntList;
 
 import com.google.common.collect.Lists;
 
+import java.util.BitSet;
+import java.util.List;
+
 /**
  * Planner rule that creates a {@code SemiJoinRule} from a
- * {@link org.eigenbase.rel.JoinRelBase} on top of a
- * {@link org.eigenbase.rel.AggregateRel}.
+ * {@link org.apache.calcite.rel.core.Join} on top of a
+ * {@link org.apache.calcite.rel.logical.LogicalAggregate}.
  */
 public class SemiJoinRule extends RelOptRule {
   public static final SemiJoinRule INSTANCE = new SemiJoinRule();
 
   private SemiJoinRule() {
     super(
-        operand(ProjectRelBase.class,
-            some(operand(JoinRelBase.class,
+        operand(Project.class,
+            some(operand(Join.class,
                 some(operand(RelNode.class, any()),
-                    operand(AggregateRelBase.class, any()))))));
+                    operand(Aggregate.class, any()))))));
   }
 
-  @Override
-  public void onMatch(RelOptRuleCall call) {
-    final ProjectRelBase project = call.rel(0);
-    final JoinRelBase join = call.rel(1);
+  @Override public void onMatch(RelOptRuleCall call) {
+    final Project project = call.rel(0);
+    final Join join = call.rel(1);
     final RelNode left = call.rel(2);
-    final AggregateRelBase aggregate = call.rel(3);
+    final Aggregate aggregate = call.rel(3);
     final BitSet bits = RelOptUtil.InputFinder.bits(project.getProjects(),
         null);
     final BitSet rightBits = BitSets.range(left.getRowType().getFieldCount(),
@@ -75,16 +74,16 @@ public class SemiJoinRule extends RelOptRule {
     for (int key : joinInfo.rightKeys) {
       newRightKeys.add(aggregateKeys.get(key));
     }
-    final SemiJoinRel semiJoin =
-        new SemiJoinRel(join.getCluster(),
+    final SemiJoin semiJoin =
+        new SemiJoin(join.getCluster(),
             join.getCluster().traitSetOf(Convention.NONE),
-            left, aggregate.getChild(),
+            left, aggregate.getInput(),
             join.getCondition(), joinInfo.leftKeys,
             ImmutableIntList.copyOf(newRightKeys));
-    final ProjectRelBase newProject =
+    final Project newProject =
         project.copy(project.getTraitSet(), semiJoin, project.getProjects(),
             project.getRowType());
-    call.transformTo(RemoveTrivialProjectRule.strip(newProject));
+    call.transformTo(ProjectRemoveRule.strip(newProject));
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/rules/SortProjectTransposeRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/SortProjectTransposeRule.java b/core/src/main/java/org/apache/calcite/rel/rules/SortProjectTransposeRule.java
index d665b19..60cfc53 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/SortProjectTransposeRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/SortProjectTransposeRule.java
@@ -14,43 +14,54 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
+package org.apache.calcite.rel.rules;
 
-import java.util.Map;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.rex.RexUtil;
-import org.eigenbase.util.mapping.Mappings;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.logical.LogicalProject;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.util.mapping.Mappings;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 
+import java.util.Map;
+
 /**
- * Planner rule that pushes a {@link SortRel} past a {@link ProjectRel}.
+ * Planner rule that pushes
+ * a {@link org.apache.calcite.rel.core.Sort}
+ * past a {@link org.apache.calcite.rel.logical.LogicalProject}.
+ *
+ * @see org.apache.calcite.rel.rules.ProjectSortTransposeRule
  */
-public class PushSortPastProjectRule extends RelOptRule {
-  public static final PushSortPastProjectRule INSTANCE =
-      new PushSortPastProjectRule();
+public class SortProjectTransposeRule extends RelOptRule {
+  public static final SortProjectTransposeRule INSTANCE =
+      new SortProjectTransposeRule();
 
   //~ Constructors -----------------------------------------------------------
 
   /**
-   * Creates a PushSortPastProjectRule.
+   * Creates a SortProjectTransposeRule.
    */
-  private PushSortPastProjectRule() {
+  private SortProjectTransposeRule() {
     super(
         operand(
-            SortRel.class,
-            operand(ProjectRel.class, any())));
+            Sort.class,
+            operand(LogicalProject.class, any())));
   }
 
   //~ Methods ----------------------------------------------------------------
 
   // implement RelOptRule
   public void onMatch(RelOptRuleCall call) {
-    final SortRel sort = call.rel(0);
-    final ProjectRel project = call.rel(1);
+    final Sort sort = call.rel(0);
+    final LogicalProject project = call.rel(1);
     final RelOptCluster cluster = project.getCluster();
 
     if (sort.getConvention() != project.getConvention()) {
@@ -61,7 +72,7 @@ public class PushSortPastProjectRule extends RelOptRule {
     // relies on non-trivial expressions, we can't push.
     final Mappings.TargetMapping map =
         RelOptUtil.permutation(
-            project.getProjects(), project.getChild().getRowType());
+            project.getProjects(), project.getInput().getRowType());
     for (RelFieldCollation fc : sort.getCollation().getFieldCollations()) {
       if (map.getTargetOpt(fc.getFieldIndex()) < 0) {
         return;
@@ -70,10 +81,10 @@ public class PushSortPastProjectRule extends RelOptRule {
     final RelCollation newCollation =
         cluster.traitSetOf().canonize(
             RexUtil.apply(map, sort.getCollation()));
-    final SortRel newSort =
+    final Sort newSort =
         sort.copy(
             sort.getTraitSet().replace(newCollation),
-            project.getChild(),
+            project.getInput(),
             newCollation,
             sort.offset,
             sort.fetch);
@@ -86,10 +97,10 @@ public class PushSortPastProjectRule extends RelOptRule {
     // (but only if the sort is not also applying an offset/limit).
     Map<RelNode, RelNode> equiv =
         sort.offset == null && sort.fetch == null
-            ? ImmutableMap.<RelNode, RelNode>of(newSort, project.getChild())
+            ? ImmutableMap.<RelNode, RelNode>of(newSort, project.getInput())
             : ImmutableMap.<RelNode, RelNode>of();
     call.transformTo(newProject, equiv);
   }
 }
 
-// End PushSortPastProjectRule.java
+// End SortProjectTransposeRule.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/rules/SortRemoveRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/SortRemoveRule.java b/core/src/main/java/org/apache/calcite/rel/rules/SortRemoveRule.java
index 03425e0..bd56821 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/SortRemoveRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/SortRemoveRule.java
@@ -14,32 +14,37 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
+package org.apache.calcite.rel.rules;
 
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.core.Sort;
 
 /**
- * Planner rule that removes a {@link SortRel} if its input is already sorted.
- * Requires {@link RelCollationTraitDef}.
+ * Planner rule that removes
+ * a {@link org.apache.calcite.rel.core.Sort} if its input is already sorted.
+ *
+ * <p>Requires {@link RelCollationTraitDef}.
  */
-public class RemoveSortRule extends RelOptRule {
-  public static final RemoveSortRule INSTANCE = new RemoveSortRule();
+public class SortRemoveRule extends RelOptRule {
+  public static final SortRemoveRule INSTANCE = new SortRemoveRule();
 
-  private RemoveSortRule() {
+  private SortRemoveRule() {
     super(
-        operand(SortRel.class, any()),
-        "RemoveSortRule");
+        operand(Sort.class, any()),
+        "SortRemoveRule");
   }
 
-  @Override
-  public void onMatch(RelOptRuleCall call) {
+  @Override public void onMatch(RelOptRuleCall call) {
     if (!call.getPlanner().getRelTraitDefs()
         .contains(RelCollationTraitDef.INSTANCE)) {
       // Collation is not an active trait.
       return;
     }
-    final SortRel sort = call.rel(0);
+    final Sort sort = call.rel(0);
     if (sort.offset != null || sort.fetch != null) {
       // Don't remove sort if would also remove OFFSET or LIMIT.
       return;
@@ -50,9 +55,9 @@ public class RemoveSortRule extends RelOptRule {
     final RelCollation collation = sort.getCollation();
     assert collation == sort.getTraitSet()
         .getTrait(RelCollationTraitDef.INSTANCE);
-    final RelTraitSet traits = sort.getChild().getTraitSet().replace(collation);
-    call.transformTo(convert(sort.getChild(), traits));
+    final RelTraitSet traits = sort.getInput().getTraitSet().replace(collation);
+    call.transformTo(convert(sort.getInput(), traits));
   }
 }
 
-// End RemoveSortRule.java
+// End SortRemoveRule.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/rules/TableScanRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/TableScanRule.java b/core/src/main/java/org/apache/calcite/rel/rules/TableScanRule.java
index 4b65808..208fc89 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/TableScanRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/TableScanRule.java
@@ -14,30 +14,35 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
+package org.apache.calcite.rel.rules;
 
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.logical.LogicalTableScan;
 
 /**
- * Planner rule that converts a {@link TableAccessRel} to the result of calling
- * {@link RelOptTable#toRel}.
+ * Planner rule that converts a
+ * {@link org.apache.calcite.rel.logical.LogicalTableScan} to the result
+ * of calling {@link RelOptTable#toRel}.
  */
-public class TableAccessRule extends RelOptRule {
+public class TableScanRule extends RelOptRule {
   //~ Static fields/initializers ---------------------------------------------
 
-  public static final TableAccessRule INSTANCE = new TableAccessRule();
+  public static final TableScanRule INSTANCE = new TableScanRule();
 
   //~ Constructors -----------------------------------------------------------
 
-  private TableAccessRule() {
-    super(operand(TableAccessRel.class, any()));
+  private TableScanRule() {
+    super(operand(LogicalTableScan.class, any()));
   }
 
   //~ Methods ----------------------------------------------------------------
 
   public void onMatch(RelOptRuleCall call) {
-    final TableAccessRel oldRel = call.rel(0);
+    final LogicalTableScan oldRel = call.rel(0);
     RelNode newRel =
         oldRel.getTable().toRel(
             RelOptUtil.getContext(oldRel.getCluster()));
@@ -45,4 +50,4 @@ public class TableAccessRule extends RelOptRule {
   }
 }
 
-// End TableAccessRule.java
+// End TableScanRule.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/rules/UnionEliminatorRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/UnionEliminatorRule.java b/core/src/main/java/org/apache/calcite/rel/rules/UnionEliminatorRule.java
index 7664b5d..f78704b 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/UnionEliminatorRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/UnionEliminatorRule.java
@@ -14,10 +14,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
+package org.apache.calcite.rel.rules;
 
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.rel.logical.LogicalUnion;
 
 /**
  * <code>UnionEliminatorRule</code> checks to see if its possible to optimize a
@@ -34,13 +35,13 @@ public class UnionEliminatorRule extends RelOptRule {
    * Creates a UnionEliminatorRule.
    */
   private UnionEliminatorRule() {
-    super(operand(UnionRel.class, any()));
+    super(operand(LogicalUnion.class, any()));
   }
 
   //~ Methods ----------------------------------------------------------------
 
   public void onMatch(RelOptRuleCall call) {
-    UnionRel union = call.rel(0);
+    LogicalUnion union = call.rel(0);
     if (union.getInputs().size() != 1) {
       return;
     }
@@ -49,7 +50,7 @@ public class UnionEliminatorRule extends RelOptRule {
     }
 
     // REVIEW jvs 14-Mar-2006:  why don't we need to register
-    // the equivalence here like we do in RemoveDistinctRule?
+    // the equivalence here like we do in AggregateRemoveRule?
 
     call.transformTo(union.getInputs().get(0));
   }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/rules/UnionMergeRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/UnionMergeRule.java b/core/src/main/java/org/apache/calcite/rel/rules/UnionMergeRule.java
index a0e3f6f..f772777 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/UnionMergeRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/UnionMergeRule.java
@@ -14,32 +14,35 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
+package org.apache.calcite.rel.rules;
+
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.logical.LogicalUnion;
+import org.apache.calcite.util.Util;
 
 import java.util.ArrayList;
 import java.util.List;
 
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.util.Util;
-
 /**
- * CombineUnionsRule implements the rule for combining two non-distinct
- * {@link UnionRel}s into a single {@link UnionRel}.
+ * UnionMergeRule implements the rule for combining two
+ * non-distinct {@link org.apache.calcite.rel.logical.LogicalUnion}s
+ * into a single {@link org.apache.calcite.rel.logical.LogicalUnion}.
  */
-public class CombineUnionsRule extends RelOptRule {
-  public static final CombineUnionsRule INSTANCE =
-      new CombineUnionsRule();
+public class UnionMergeRule extends RelOptRule {
+  public static final UnionMergeRule INSTANCE =
+      new UnionMergeRule();
 
   //~ Constructors -----------------------------------------------------------
 
   /**
-   * Creates a CombineUnionsRule.
+   * Creates a UnionMergeRule.
    */
-  private CombineUnionsRule() {
+  private UnionMergeRule() {
     super(
         operand(
-            UnionRel.class,
+            LogicalUnion.class,
             operand(RelNode.class, any()),
             operand(RelNode.class, any())));
   }
@@ -48,50 +51,50 @@ public class CombineUnionsRule extends RelOptRule {
 
   // implement RelOptRule
   public void onMatch(RelOptRuleCall call) {
-    UnionRel topUnionRel = call.rel(0);
-    UnionRel bottomUnionRel;
+    LogicalUnion topUnion = call.rel(0);
+    LogicalUnion bottomUnion;
 
-    // We want to combine the UnionRel that's in the second input first.
+    // We want to combine the Union that's in the second input first.
     // Hence, that's why the rule pattern matches on generic RelNodes
     // rather than explicit UnionRels.  By doing so, and firing this rule
     // in a bottom-up order, it allows us to only specify a single
     // pattern for this rule.
-    if (call.rel(2) instanceof UnionRel) {
-      bottomUnionRel = call.rel(2);
-    } else if (call.rel(1) instanceof UnionRel) {
-      bottomUnionRel = call.rel(1);
+    if (call.rel(2) instanceof LogicalUnion) {
+      bottomUnion = call.rel(2);
+    } else if (call.rel(1) instanceof LogicalUnion) {
+      bottomUnion = call.rel(1);
     } else {
       return;
     }
 
     // If distincts haven't been removed yet, defer invoking this rule
-    if (!topUnionRel.all || !bottomUnionRel.all) {
+    if (!topUnion.all || !bottomUnion.all) {
       return;
     }
 
     // Combine the inputs from the bottom union with the other inputs from
     // the top union
     List<RelNode> unionInputs = new ArrayList<RelNode>();
-    if (call.rel(2) instanceof UnionRel) {
-      assert topUnionRel.getInputs().size() == 2;
-      unionInputs.add(topUnionRel.getInput(0));
-      unionInputs.addAll(bottomUnionRel.getInputs());
+    if (call.rel(2) instanceof LogicalUnion) {
+      assert topUnion.getInputs().size() == 2;
+      unionInputs.add(topUnion.getInput(0));
+      unionInputs.addAll(bottomUnion.getInputs());
     } else {
-      unionInputs.addAll(bottomUnionRel.getInputs());
-      unionInputs.addAll(Util.skip(topUnionRel.getInputs()));
+      unionInputs.addAll(bottomUnion.getInputs());
+      unionInputs.addAll(Util.skip(topUnion.getInputs()));
     }
     assert unionInputs.size()
-        == bottomUnionRel.getInputs().size()
-        + topUnionRel.getInputs().size()
+        == bottomUnion.getInputs().size()
+        + topUnion.getInputs().size()
         - 1;
-    UnionRel newUnionRel =
-        new UnionRel(
-            topUnionRel.getCluster(),
+    LogicalUnion newUnion =
+        new LogicalUnion(
+            topUnion.getCluster(),
             unionInputs,
             true);
 
-    call.transformTo(newUnionRel);
+    call.transformTo(newUnion);
   }
 }
 
-// End CombineUnionsRule.java
+// End UnionMergeRule.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/rules/UnionToDistinctRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/UnionToDistinctRule.java b/core/src/main/java/org/apache/calcite/rel/rules/UnionToDistinctRule.java
index 9a3227a..39a6d92 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/UnionToDistinctRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/UnionToDistinctRule.java
@@ -14,15 +14,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
+package org.apache.calcite.rel.rules;
 
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.logical.LogicalUnion;
 
 /**
- * <code>UnionToDistinctRule</code> translates a distinct {@link UnionRel}
- * (<code>all</code> = <code>false</code>) into an {@link AggregateRel} on top
- * of a non-distinct {@link UnionRel} (<code>all</code> = <code>true</code>).
+ * Planner rule that translates a distinct
+ * {@link org.apache.calcite.rel.logical.LogicalUnion}
+ * (<code>all</code> = <code>false</code>)
+ * into an {@link org.apache.calcite.rel.logical.LogicalAggregate}
+ * on top of a non-distinct {@link org.apache.calcite.rel.logical.LogicalUnion}
+ * (<code>all</code> = <code>true</code>).
  */
 public class UnionToDistinctRule extends RelOptRule {
   public static final UnionToDistinctRule INSTANCE =
@@ -34,18 +39,18 @@ public class UnionToDistinctRule extends RelOptRule {
    * Creates a UnionToDistinctRule.
    */
   private UnionToDistinctRule() {
-    super(operand(UnionRel.class, any()));
+    super(operand(LogicalUnion.class, any()));
   }
 
   //~ Methods ----------------------------------------------------------------
 
   public void onMatch(RelOptRuleCall call) {
-    UnionRel union = call.rel(0);
+    LogicalUnion union = call.rel(0);
     if (union.all) {
       return; // nothing to do
     }
-    UnionRel unionAll =
-        new UnionRel(
+    LogicalUnion unionAll =
+        new LogicalUnion(
             union.getCluster(),
             union.getInputs(),
             true);

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/rules/ValuesReduceRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/ValuesReduceRule.java b/core/src/main/java/org/apache/calcite/rel/rules/ValuesReduceRule.java
index cded234..d3f8d6f 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/ValuesReduceRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/ValuesReduceRule.java
@@ -14,22 +14,37 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
+package org.apache.calcite.rel.rules;
+
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptRuleOperand;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Empty;
+import org.apache.calcite.rel.logical.LogicalFilter;
+import org.apache.calcite.rel.logical.LogicalProject;
+import org.apache.calcite.rel.logical.LogicalValues;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexShuttle;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.util.Util;
+import org.apache.calcite.util.trace.CalciteTrace;
 
 import java.util.ArrayList;
 import java.util.List;
 import java.util.logging.Logger;
 
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.rex.*;
-import org.eigenbase.trace.EigenbaseTrace;
-import org.eigenbase.util.Util;
-
 /**
  * Planner rule that folds projections and filters into an underlying
- * {@link ValuesRel}. Returns an {@link EmptyRel} if all rows are filtered away.
+ * {@link org.apache.calcite.rel.logical.LogicalValues}.
+ *
+ * <p>Returns a simplified {@code Values},
+ * or an {@link org.apache.calcite.rel.core.Empty} if all rows are
+ * filtered away.
  *
  * <p>For example,</p>
  *
@@ -40,23 +55,23 @@ import org.eigenbase.util.Util;
  *
  * <blockquote><code>select x from (values (-2), (-4))</code></blockquote>
  */
-public abstract class ReduceValuesRule extends RelOptRule {
+public abstract class ValuesReduceRule extends RelOptRule {
   //~ Static fields/initializers ---------------------------------------------
 
-  private static final Logger LOGGER = EigenbaseTrace.getPlannerTracer();
+  private static final Logger LOGGER = CalciteTrace.getPlannerTracer();
 
   /**
    * Instance of this rule that applies to the pattern
    * Filter(Values).
    */
-  public static final ReduceValuesRule FILTER_INSTANCE =
-      new ReduceValuesRule(
-          operand(FilterRel.class,
-              operand(ValuesRel.class, none())),
-          "ReduceValuesRule[Filter") {
+  public static final ValuesReduceRule FILTER_INSTANCE =
+      new ValuesReduceRule(
+          operand(LogicalFilter.class,
+              operand(LogicalValues.class, none())),
+          "ValuesReduceRule[Filter") {
         public void onMatch(RelOptRuleCall call) {
-          FilterRel filter = call.rel(0);
-          ValuesRel values = call.rel(1);
+          LogicalFilter filter = call.rel(0);
+          LogicalValues values = call.rel(1);
           apply(call, null, filter, values);
         }
       };
@@ -65,14 +80,14 @@ public abstract class ReduceValuesRule extends RelOptRule {
    * Instance of this rule that applies to the pattern
    * Project(Values).
    */
-  public static final ReduceValuesRule PROJECT_INSTANCE =
-      new ReduceValuesRule(
-          operand(ProjectRel.class,
-              operand(ValuesRel.class, none())),
-          "ReduceValuesRule[Project]") {
+  public static final ValuesReduceRule PROJECT_INSTANCE =
+      new ValuesReduceRule(
+          operand(LogicalProject.class,
+              operand(LogicalValues.class, none())),
+          "ValuesReduceRule[Project]") {
         public void onMatch(RelOptRuleCall call) {
-          ProjectRel project = call.rel(0);
-          ValuesRel values = call.rel(1);
+          LogicalProject project = call.rel(0);
+          LogicalValues values = call.rel(1);
           apply(call, project, null, values);
         }
       };
@@ -81,16 +96,16 @@ public abstract class ReduceValuesRule extends RelOptRule {
    * Singleton instance of this rule that applies to the pattern
    * Project(Filter(Values)).
    */
-  public static final ReduceValuesRule PROJECT_FILTER_INSTANCE =
-      new ReduceValuesRule(
-          operand(ProjectRel.class,
-              operand(FilterRel.class,
-                  operand(ValuesRel.class, none()))),
-          "ReduceValuesRule[Project+Filter]") {
+  public static final ValuesReduceRule PROJECT_FILTER_INSTANCE =
+      new ValuesReduceRule(
+          operand(LogicalProject.class,
+              operand(LogicalFilter.class,
+                  operand(LogicalValues.class, none()))),
+          "ValuesReduceRule[Project+Filter]") {
         public void onMatch(RelOptRuleCall call) {
-          ProjectRel project = call.rel(0);
-          FilterRel filter = call.rel(1);
-          ValuesRel values = call.rel(2);
+          LogicalProject project = call.rel(0);
+          LogicalFilter filter = call.rel(1);
+          LogicalValues values = call.rel(2);
           apply(call, project, filter, values);
         }
       };
@@ -98,11 +113,11 @@ public abstract class ReduceValuesRule extends RelOptRule {
   //~ Constructors -----------------------------------------------------------
 
   /**
-   * Creates a ReduceValuesRule.
+   * Creates a ValuesReduceRule.
    *
    * @param operand class of rels to which this rule should apply
    */
-  private ReduceValuesRule(RelOptRuleOperand operand, String desc) {
+  private ValuesReduceRule(RelOptRuleOperand operand, String desc) {
     super(operand, desc);
     Util.discard(LOGGER);
   }
@@ -117,8 +132,8 @@ public abstract class ReduceValuesRule extends RelOptRule {
    * @param filter  Filter, may be null
    * @param values  Values rel to be reduced
    */
-  protected void apply(RelOptRuleCall call, ProjectRel project,
-      FilterRel filter, ValuesRel values) {
+  protected void apply(RelOptRuleCall call, LogicalProject project,
+      LogicalFilter filter, LogicalValues values) {
     assert values != null;
     assert filter != null || project != null;
     final RexNode conditionExpr =
@@ -203,12 +218,12 @@ public abstract class ReduceValuesRule extends RelOptRule {
       final RelNode newRel;
       if (tupleList.isEmpty()) {
         newRel =
-            new EmptyRel(
+            new Empty(
                 values.getCluster(),
                 rowType);
       } else {
         newRel =
-            new ValuesRel(
+            new LogicalValues(
                 values.getCluster(),
                 rowType,
                 tupleList);
@@ -241,4 +256,4 @@ public abstract class ReduceValuesRule extends RelOptRule {
   }
 }
 
-// End ReduceValuesRule.java
+// End ValuesReduceRule.java


[27/58] [abbrv] [partial] incubator-calcite git commit: [CALCITE-306] Standardize code style for "import package.*; "

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/SqlCharStringLiteral.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlCharStringLiteral.java b/core/src/main/java/org/apache/calcite/sql/SqlCharStringLiteral.java
index d18a5f5..9f15a22 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlCharStringLiteral.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlCharStringLiteral.java
@@ -14,17 +14,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
+package org.apache.calcite.sql;
 
-import java.util.List;
-
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.util.*;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.util.Bug;
+import org.apache.calcite.util.NlsString;
+import org.apache.calcite.util.Util;
 
 import com.google.common.base.Function;
 import com.google.common.collect.Lists;
 
+import java.util.List;
+
 /**
  * A character string literal.
  *

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/SqlCollation.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlCollation.java b/core/src/main/java/org/apache/calcite/sql/SqlCollation.java
index 7ff8ce1..947e48c 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlCollation.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlCollation.java
@@ -14,19 +14,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
+package org.apache.calcite.sql;
 
-import java.io.*;
+import org.apache.calcite.sql.parser.SqlParserUtil;
+import org.apache.calcite.util.SaffronProperties;
+import org.apache.calcite.util.SerializableCharset;
+import org.apache.calcite.util.Util;
 
-import java.nio.charset.*;
+import java.io.Serializable;
+import java.nio.charset.Charset;
+import java.util.Locale;
 
-import java.util.*;
-
-import org.eigenbase.resource.*;
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.util.*;
-
-import static org.eigenbase.util.Static.RESOURCE;
+import static org.apache.calcite.util.Static.RESOURCE;
 
 /**
  * A <code>SqlCollation</code> is an object representing a <code>Collate</code>
@@ -115,8 +114,7 @@ public class SqlCollation implements Serializable {
         && collationName.equals(((SqlCollation) o).collationName);
   }
 
-  @Override
-  public int hashCode() {
+  @Override public int hashCode() {
     return collationName.hashCode();
   }
 
@@ -144,9 +142,11 @@ public class SqlCollation implements Serializable {
    * @param col1 first operand for the dyadic operation
    * @param col2 second operand for the dyadic operation
    * @return the resulting collation sequence
-   * @throws EigenbaseException {@link EigenbaseNewResource#invalidCompare} or
-   *                            {@link EigenbaseNewResource#differentCollations}
-   *                            if no collating sequence can be deduced
+   *
+   * @throws org.apache.calcite.runtime.CalciteException
+   *   {@link org.apache.calcite.runtime.CalciteResource#invalidCompare} or
+   *   {@link org.apache.calcite.runtime.CalciteResource#differentCollations}
+   *   if no collating sequence can be deduced
    *
    * @sql.99 Part 2 Section 4.2.3 Table 2
    */
@@ -170,8 +170,10 @@ public class SqlCollation implements Serializable {
    *
    * @param col1 first operand for the dyadic operation
    * @param col2 second operand for the dyadic operation
-   * @return the resulting collation sequence. If no collating sequence could
-   * be deduced a {@link EigenbaseNewResource#invalidCompare} is thrown
+   *
+   * @return the resulting collation sequence. If no collating
+   * sequence could be deduced throws a
+   * {@link org.apache.calcite.runtime.CalciteResource#invalidCompare}
    *
    * @sql.99 Part 2 Section 4.2.3 Table 3
    */

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 deb2a38..e366204 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlDataTypeSpec.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlDataTypeSpec.java
@@ -14,20 +14,25 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
+package org.apache.calcite.sql;
 
-import java.nio.charset.*;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.type.SqlTypeUtil;
+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.Util;
 
-import java.util.*;
+import com.google.common.base.Objects;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.sql.util.*;
-import org.eigenbase.sql.validate.*;
-import org.eigenbase.util.*;
+import java.nio.charset.Charset;
+import java.util.TimeZone;
 
-import static org.eigenbase.util.Static.RESOURCE;
+import static org.apache.calcite.util.Static.RESOURCE;
 
 /**
  * Represents a SQL data type specification in a parse tree.
@@ -222,11 +227,12 @@ public class SqlDataTypeSpec extends SqlNode {
       assert !fail : this + "!=" + node;
       return false;
     }
-    if (!Util.equal(this.timeZone, that.timeZone)) {
+    if (!Objects.equal(this.timeZone, that.timeZone)) {
       assert !fail : this + "!=" + node;
       return false;
     }
-    if (!Util.equal(this.charSetName, that.charSetName)) {
+    if (!com.google.common.base.Objects.equal(this.charSetName,
+        that.charSetName)) {
       assert !fail : this + "!=" + node;
       return false;
     }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/SqlDateLiteral.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlDateLiteral.java b/core/src/main/java/org/apache/calcite/sql/SqlDateLiteral.java
index a3d2eb9..9486c58 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlDateLiteral.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlDateLiteral.java
@@ -14,14 +14,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
+package org.apache.calcite.sql;
 
-import java.util.*;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.util.DateTimeUtil;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.util14.DateTimeUtil;
+import java.util.Calendar;
 
 /**
  * A SQL literal representing a DATE value, such as <code>DATE

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/SqlDelete.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlDelete.java b/core/src/main/java/org/apache/calcite/sql/SqlDelete.java
index 3533c99..5401e2b 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlDelete.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlDelete.java
@@ -14,13 +14,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
+package org.apache.calcite.sql;
 
-import java.util.List;
+import org.apache.calcite.sql.parser.SqlParserPos;
+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.ImmutableNullableList;
 
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.sql.validate.*;
-import org.eigenbase.util.ImmutableNullableList;
+import java.util.List;
 
 /**
  * A <code>SqlDelete</code> is a node of a parse tree which represents a DELETE

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/SqlDialect.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlDialect.java b/core/src/main/java/org/apache/calcite/sql/SqlDialect.java
index 0a3102b..931d463 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlDialect.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlDialect.java
@@ -14,19 +14,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
-
-import java.sql.*;
+package org.apache.calcite.sql;
 
+import java.sql.DatabaseMetaData;
+import java.sql.SQLException;
+import java.sql.Timestamp;
 import java.text.SimpleDateFormat;
-import java.util.*;
-import java.util.regex.*;
+import java.util.List;
+import java.util.TimeZone;
+import java.util.regex.Pattern;
 
 /**
  * <code>SqlDialect</code> encapsulates the differences between dialects of SQL.
  *
  * <p>It is used by classes such as {@link SqlWriter} and
- * {@link org.eigenbase.sql.util.SqlBuilder}.
+ * {@link org.apache.calcite.sql.util.SqlBuilder}.
  */
 public class SqlDialect {
   //~ Static fields/initializers ---------------------------------------------
@@ -41,12 +43,12 @@ public class SqlDialect {
 
   /**
    * A dialect useful for generating SQL which can be parsed by the
-   * Eigenbase parser, in particular quoting literals and identifiers. If you
+   * Calcite parser, in particular quoting literals and identifiers. If you
    * want a dialect that knows the full capabilities of the database, create
    * one from a connection.
    */
-  public static final SqlDialect EIGENBASE =
-      DatabaseProduct.LUCIDDB.getDialect();
+  public static final SqlDialect CALCITE =
+      DatabaseProduct.CALCITE.getDialect();
 
   //~ Instance fields --------------------------------------------------------
 
@@ -414,7 +416,7 @@ public class SqlDialect {
   }
 
   /**
-   * A few utility functions copied from org.eigenbase.util.Util. We have
+   * A few utility functions copied from org.apache.calcite.util.Util. We have
    * copied them because we wish to keep SqlDialect's dependencies to a
    * minimum.
    */
@@ -473,6 +475,7 @@ public class SqlDialect {
    */
   public enum DatabaseProduct {
     ACCESS("Access", "\""),
+    CALCITE("Apache Calcite", "\""),
     MSSQL("Microsoft SQL Server", "["),
     MYSQL("MySQL", "`"),
     ORACLE("Oracle", "\""),
@@ -483,7 +486,6 @@ public class SqlDialect {
     INFORMIX("Informix", null),
     INGRES("Ingres", null),
     LUCIDDB("LucidDB", "\""),
-    OPTIQ("Apache Calcite", "\""),
     INTERBASE("Interbase", null),
     PHOENIX("Phoenix", "\""),
     POSTGRESQL("PostgreSQL", "\""),

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 4575c55..552852c 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlDynamicParam.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlDynamicParam.java
@@ -14,11 +14,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
+package org.apache.calcite.sql;
 
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.sql.util.*;
-import org.eigenbase.sql.validate.*;
+import org.apache.calcite.sql.parser.SqlParserPos;
+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;
 
 /**
  * A <code>SqlDynamicParam</code> represents a dynamic parameter marker in an

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/SqlExplain.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlExplain.java b/core/src/main/java/org/apache/calcite/sql/SqlExplain.java
index 1de39f1..7004caf 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlExplain.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlExplain.java
@@ -14,12 +14,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
+package org.apache.calcite.sql;
 
-import java.util.List;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.util.ImmutableNullableList;
 
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.util.ImmutableNullableList;
+import java.util.List;
 
 /**
  * A <code>SqlExplain</code> is a node of a parse tree which represents an
@@ -28,8 +28,7 @@ import org.eigenbase.util.ImmutableNullableList;
 public class SqlExplain extends SqlCall {
   public static final SqlSpecialOperator OPERATOR =
       new SqlSpecialOperator("EXPLAIN", SqlKind.EXPLAIN) {
-        @Override
-        public SqlCall createCall(SqlLiteral functionQualifier,
+        @Override public SqlCall createCall(SqlLiteral functionQualifier,
             SqlParserPos pos, SqlNode... operands) {
           return new SqlExplain(pos, operands[0], (SqlLiteral) operands[1],
               (SqlLiteral) operands[2], (SqlLiteral) operands[3], 0);

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/SqlExplainLevel.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlExplainLevel.java b/core/src/main/java/org/apache/calcite/sql/SqlExplainLevel.java
index 21616fd..4186393 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlExplainLevel.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlExplainLevel.java
@@ -14,9 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
+package org.apache.calcite.sql;
 
-import org.eigenbase.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.parser.SqlParserPos;
 
 /**
  * SqlExplainLevel defines detail levels for EXPLAIN PLAN.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/SqlFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlFunction.java b/core/src/main/java/org/apache/calcite/sql/SqlFunction.java
index 14fb506..6e5d99a 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlFunction.java
@@ -14,19 +14,24 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
-
-import java.util.List;
-
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.sql.validate.*;
-import org.eigenbase.util.Util;
+package org.apache.calcite.sql;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.SqlOperandTypeChecker;
+import org.apache.calcite.sql.type.SqlOperandTypeInference;
+import org.apache.calcite.sql.type.SqlReturnTypeInference;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql.validate.SqlValidatorScope;
+import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
 
-import static org.eigenbase.util.Static.RESOURCE;
+import java.util.List;
+
+import static org.apache.calcite.util.Static.RESOURCE;
 
 /**
  * A <code>SqlFunction</code> is a type of operator which has conventional

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/SqlFunctionCategory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlFunctionCategory.java b/core/src/main/java/org/apache/calcite/sql/SqlFunctionCategory.java
index 90a76f0..8c836da 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlFunctionCategory.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlFunctionCategory.java
@@ -14,9 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
+package org.apache.calcite.sql;
 
-import org.eigenbase.util.*;
+import org.apache.calcite.util.Util;
 
 /**
  * Enumeration of the categories of

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/SqlFunctionalOperator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlFunctionalOperator.java b/core/src/main/java/org/apache/calcite/sql/SqlFunctionalOperator.java
index a56cfda..e1f1517 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlFunctionalOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlFunctionalOperator.java
@@ -14,9 +14,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
+package org.apache.calcite.sql;
 
-import org.eigenbase.sql.type.*;
+import org.apache.calcite.sql.type.SqlOperandTypeChecker;
+import org.apache.calcite.sql.type.SqlOperandTypeInference;
+import org.apache.calcite.sql.type.SqlReturnTypeInference;
 
 /**
  * SqlFunctionalOperator is a base class for special operators which use

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 5231c69..d471c85 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlIdentifier.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlIdentifier.java
@@ -14,17 +14,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
+package org.apache.calcite.sql;
 
-import java.util.*;
-
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.sql.util.*;
-import org.eigenbase.sql.validate.*;
-import org.eigenbase.util.Util;
+import org.apache.calcite.sql.parser.SqlParserPos;
+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.SqlValidatorNamespace;
+import org.apache.calcite.sql.validate.SqlValidatorScope;
+import org.apache.calcite.sql.validate.SqlValidatorUtil;
+import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
 
+import java.util.List;
+
 /**
  * A <code>SqlIdentifier</code> is an identifier, possibly compound.
  */
@@ -150,8 +154,9 @@ public class SqlIdentifier extends SqlNode {
 
   /**
    * Creates an identifier which contains only the <code>ordinal</code>th
-   * component of this compound identifier. It will have the correct {@link
-   * SqlParserPos}, provided that detailed position information is available.
+   * component of this compound identifier. It will have the correct
+   * {@link SqlParserPos}, provided that detailed position information is
+   * available.
    */
   public SqlIdentifier getComponent(int ordinal) {
     return new SqlIdentifier(names.get(ordinal),

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/SqlInfixOperator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlInfixOperator.java b/core/src/main/java/org/apache/calcite/sql/SqlInfixOperator.java
index 20ea130..2f26878 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlInfixOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlInfixOperator.java
@@ -14,11 +14,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
+package org.apache.calcite.sql;
 
-import org.eigenbase.sql.type.*;
-
-import net.hydromatic.linq4j.Ord;
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.sql.type.SqlOperandTypeChecker;
+import org.apache.calcite.sql.type.SqlOperandTypeInference;
+import org.apache.calcite.sql.type.SqlReturnTypeInference;
 
 /**
  * A generalization of a binary operator to involve several (two or more)

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/SqlInsert.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlInsert.java b/core/src/main/java/org/apache/calcite/sql/SqlInsert.java
index 6a917b7..71132b0 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlInsert.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlInsert.java
@@ -14,13 +14,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
+package org.apache.calcite.sql;
 
-import java.util.List;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql.validate.SqlValidatorScope;
+import org.apache.calcite.util.ImmutableNullableList;
 
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.sql.validate.*;
-import org.eigenbase.util.ImmutableNullableList;
+import java.util.List;
 
 /**
  * A <code>SqlInsert</code> is a node of a parse tree which represents an INSERT

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/SqlInsertKeyword.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlInsertKeyword.java b/core/src/main/java/org/apache/calcite/sql/SqlInsertKeyword.java
index 9058102..8e93954 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlInsertKeyword.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlInsertKeyword.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
+package org.apache.calcite.sql;
 
 /**
  * Defines the keywords which can occur immediately after the "INSERT" keyword.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/SqlInternalOperator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlInternalOperator.java b/core/src/main/java/org/apache/calcite/sql/SqlInternalOperator.java
index dcf774b..4eebe26 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlInternalOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlInternalOperator.java
@@ -14,9 +14,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
+package org.apache.calcite.sql;
 
-import org.eigenbase.sql.type.*;
+import org.apache.calcite.sql.type.SqlOperandTypeChecker;
+import org.apache.calcite.sql.type.SqlOperandTypeInference;
+import org.apache.calcite.sql.type.SqlReturnTypeInference;
 
 /**
  * Generic operator for nodes with internal syntax.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/SqlIntervalLiteral.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlIntervalLiteral.java b/core/src/main/java/org/apache/calcite/sql/SqlIntervalLiteral.java
index fca049e..119c73e 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlIntervalLiteral.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlIntervalLiteral.java
@@ -14,11 +14,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
+package org.apache.calcite.sql;
 
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.util.*;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.util.Util;
 
 /**
  * A SQL literal representing a time interval.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 406ed57..1d036a7 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlIntervalQualifier.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlIntervalQualifier.java
@@ -14,27 +14,30 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
-
-import java.math.BigDecimal;
-import java.util.*;
-import java.util.regex.*;
-
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.reltype.RelDataTypeSystem;
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.sql.util.*;
-import org.eigenbase.sql.validate.*;
-import org.eigenbase.util.*;
-import org.eigenbase.util14.DateTimeUtil;
-
-import net.hydromatic.optiq.runtime.SqlFunctions;
+package org.apache.calcite.sql;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.runtime.CalciteContextException;
+import org.apache.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.parser.SqlParserPos;
+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.DateTimeUtil;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableMap;
 
-import static org.eigenbase.util.Static.RESOURCE;
+import java.math.BigDecimal;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import static org.apache.calcite.util.Static.RESOURCE;
 
 /**
  * Represents an INTERVAL qualifier.
@@ -147,6 +150,7 @@ public class SqlIntervalQualifier extends SqlNode {
     }
   }
 
+  /** Range of time units. */
   private enum TimeUnitRange {
     YEAR(TimeUnit.YEAR, null),
     YEAR_TO_MONTH(TimeUnit.YEAR, TimeUnit.MONTH),
@@ -566,7 +570,8 @@ public class SqlIntervalQualifier extends SqlNode {
   /**
    * Validates an INTERVAL literal against a YEAR interval qualifier.
    *
-   * @throws EigenbaseContextException if the interval value is illegal.
+   * @throws org.apache.calcite.runtime.CalciteContextException
+   * if the interval value is illegal
    */
   private int[] evaluateIntervalLiteralAsYear(
       RelDataTypeSystem typeSystem, int sign,
@@ -600,7 +605,8 @@ public class SqlIntervalQualifier extends SqlNode {
   /**
    * Validates an INTERVAL literal against a YEAR TO MONTH interval qualifier.
    *
-   * @throws EigenbaseContextException if the interval value is illegal.
+   * @throws org.apache.calcite.runtime.CalciteContextException
+   * if the interval value is illegal
    */
   private int[] evaluateIntervalLiteralAsYearToMonth(
       RelDataTypeSystem typeSystem, int sign,
@@ -639,7 +645,8 @@ public class SqlIntervalQualifier extends SqlNode {
   /**
    * Validates an INTERVAL literal against a MONTH interval qualifier.
    *
-   * @throws EigenbaseContextException if the interval value is illegal.
+   * @throws org.apache.calcite.runtime.CalciteContextException
+   * if the interval value is illegal
    */
   private int[] evaluateIntervalLiteralAsMonth(
       RelDataTypeSystem typeSystem, int sign,
@@ -673,7 +680,8 @@ public class SqlIntervalQualifier extends SqlNode {
   /**
    * Validates an INTERVAL literal against a DAY interval qualifier.
    *
-   * @throws EigenbaseContextException if the interval value is illegal.
+   * @throws org.apache.calcite.runtime.CalciteContextException
+   * if the interval value is illegal
    */
   private int[] evaluateIntervalLiteralAsDay(
       RelDataTypeSystem typeSystem, int sign,
@@ -707,7 +715,8 @@ public class SqlIntervalQualifier extends SqlNode {
   /**
    * Validates an INTERVAL literal against a DAY TO HOUR interval qualifier.
    *
-   * @throws EigenbaseContextException if the interval value is illegal.
+   * @throws org.apache.calcite.runtime.CalciteContextException
+   * if the interval value is illegal
    */
   private int[] evaluateIntervalLiteralAsDayToHour(
       RelDataTypeSystem typeSystem, int sign,
@@ -746,7 +755,8 @@ public class SqlIntervalQualifier extends SqlNode {
   /**
    * Validates an INTERVAL literal against a DAY TO MINUTE interval qualifier.
    *
-   * @throws EigenbaseContextException if the interval value is illegal.
+   * @throws org.apache.calcite.runtime.CalciteContextException
+   * if the interval value is illegal
    */
   private int[] evaluateIntervalLiteralAsDayToMinute(
       RelDataTypeSystem typeSystem, int sign,
@@ -788,7 +798,8 @@ public class SqlIntervalQualifier extends SqlNode {
   /**
    * Validates an INTERVAL literal against a DAY TO SECOND interval qualifier.
    *
-   * @throws EigenbaseContextException if the interval value is illegal.
+   * @throws org.apache.calcite.runtime.CalciteContextException
+   * if the interval value is illegal
    */
   private int[] evaluateIntervalLiteralAsDayToSecond(
       RelDataTypeSystem typeSystem, int sign,
@@ -863,7 +874,8 @@ public class SqlIntervalQualifier extends SqlNode {
   /**
    * Validates an INTERVAL literal against an HOUR interval qualifier.
    *
-   * @throws EigenbaseContextException if the interval value is illegal.
+   * @throws org.apache.calcite.runtime.CalciteContextException
+   * if the interval value is illegal
    */
   private int[] evaluateIntervalLiteralAsHour(
       RelDataTypeSystem typeSystem, int sign,
@@ -898,7 +910,8 @@ public class SqlIntervalQualifier extends SqlNode {
    * Validates an INTERVAL literal against an HOUR TO MINUTE interval
    * qualifier.
    *
-   * @throws EigenbaseContextException if the interval value is illegal.
+   * @throws org.apache.calcite.runtime.CalciteContextException
+   * if the interval value is illegal
    */
   private int[] evaluateIntervalLiteralAsHourToMinute(
       RelDataTypeSystem typeSystem, int sign,
@@ -938,7 +951,8 @@ public class SqlIntervalQualifier extends SqlNode {
    * Validates an INTERVAL literal against an HOUR TO SECOND interval
    * qualifier.
    *
-   * @throws EigenbaseContextException if the interval value is illegal.
+   * @throws org.apache.calcite.runtime.CalciteContextException
+   * if the interval value is illegal
    */
   private int[] evaluateIntervalLiteralAsHourToSecond(
       RelDataTypeSystem typeSystem, int sign,
@@ -1010,7 +1024,8 @@ public class SqlIntervalQualifier extends SqlNode {
   /**
    * Validates an INTERVAL literal against an MINUTE interval qualifier.
    *
-   * @throws EigenbaseContextException if the interval value is illegal.
+   * @throws org.apache.calcite.runtime.CalciteContextException
+   * if the interval value is illegal
    */
   private int[] evaluateIntervalLiteralAsMinute(
       RelDataTypeSystem typeSystem, int sign,
@@ -1045,7 +1060,8 @@ public class SqlIntervalQualifier extends SqlNode {
    * Validates an INTERVAL literal against an MINUTE TO SECOND interval
    * qualifier.
    *
-   * @throws EigenbaseContextException if the interval value is illegal.
+   * @throws org.apache.calcite.runtime.CalciteContextException
+   * if the interval value is illegal
    */
   private int[] evaluateIntervalLiteralAsMinuteToSecond(
       RelDataTypeSystem typeSystem, int sign,
@@ -1113,7 +1129,8 @@ public class SqlIntervalQualifier extends SqlNode {
   /**
    * Validates an INTERVAL literal against an SECOND interval qualifier.
    *
-   * @throws EigenbaseContextException if the interval value is illegal.
+   * @throws org.apache.calcite.runtime.CalciteContextException
+   * if the interval value is illegal
    */
   private int[] evaluateIntervalLiteralAsSecond(
       RelDataTypeSystem typeSystem,
@@ -1178,7 +1195,9 @@ public class SqlIntervalQualifier extends SqlNode {
    * invalid qualifier could lead to strange results.
    *
    * @return field values, never null
-   * @throws EigenbaseContextException if the interval value is illegal
+   *
+   * @throws org.apache.calcite.runtime.CalciteContextException
+   * if the interval value is illegal
    */
   public int[] evaluateIntervalLiteral(String value, SqlParserPos pos,
       RelDataTypeSystem typeSystem) {
@@ -1251,14 +1270,14 @@ public class SqlIntervalQualifier extends SqlNode {
     return new BigDecimal(m.group(i));
   }
 
-  private EigenbaseContextException invalidValueException(SqlParserPos pos,
+  private CalciteContextException invalidValueException(SqlParserPos pos,
       String value) {
     return SqlUtil.newContextException(pos,
         RESOURCE.unsupportedIntervalLiteral(
             "'" + value + "'", "INTERVAL " + toString()));
   }
 
-  private EigenbaseContextException fieldExceedsPrecisionException(
+  private CalciteContextException fieldExceedsPrecisionException(
       SqlParserPos pos, int sign, BigDecimal value, TimeUnit type,
       int precision) {
     if (sign == -1) {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/SqlJdbcFunctionCall.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlJdbcFunctionCall.java b/core/src/main/java/org/apache/calcite/sql/SqlJdbcFunctionCall.java
index 8712aa2..91a8acf 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlJdbcFunctionCall.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlJdbcFunctionCall.java
@@ -14,18 +14,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
+package org.apache.calcite.sql;
 
-import java.util.*;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.fun.SqlTrimFunction;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql.validate.SqlValidatorScope;
+import org.apache.calcite.util.Util;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.fun.*;
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.sql.validate.*;
-import org.eigenbase.util.*;
+import java.util.HashMap;
+import java.util.Map;
 
-import static org.eigenbase.util.Static.RESOURCE;
+import static org.apache.calcite.util.Static.RESOURCE;
 
 /**
  * A <code>SqlJdbcFunctionCall</code> is a node of a parse tree which represents
@@ -655,7 +658,7 @@ public class SqlJdbcFunctionCall extends SqlFunction {
    */
   private static class JdbcToInternalLookupTable {
     /**
-     * The {@link org.eigenbase.util.Glossary#SINGLETON_PATTERN singleton}
+     * The {@link org.apache.calcite.util.Glossary#SINGLETON_PATTERN singleton}
      * instance.
      */
     static final JdbcToInternalLookupTable INSTANCE =
@@ -707,8 +710,7 @@ public class SqlJdbcFunctionCall extends SqlFunction {
       map.put(
           "LTRIM",
           new MakeCall(SqlStdOperatorTable.TRIM, 1) {
-            @Override
-            SqlCall createCall(
+            @Override SqlCall createCall(
                 SqlNode[] operands, SqlParserPos pos) {
               assert 1 == operands.length;
               return super.createCall(
@@ -723,8 +725,7 @@ public class SqlJdbcFunctionCall extends SqlFunction {
       map.put(
           "RTRIM",
           new MakeCall(SqlStdOperatorTable.TRIM, 1) {
-            @Override
-            SqlCall createCall(
+            @Override SqlCall createCall(
                 SqlNode[] operands, SqlParserPos pos) {
               assert 1 == operands.length;
               return super.createCall(

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/SqlJoin.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlJoin.java b/core/src/main/java/org/apache/calcite/sql/SqlJoin.java
index a86fabd..fc909c4 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlJoin.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlJoin.java
@@ -14,14 +14,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
+package org.apache.calcite.sql;
 
-import java.util.List;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.util.ImmutableNullableList;
+import org.apache.calcite.util.Util;
 
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.sql.type.SqlTypeName;
-import org.eigenbase.util.ImmutableNullableList;
-import org.eigenbase.util.Util;
+import java.util.List;
 
 /**
  * Parse tree node representing a {@code JOIN} clause.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/SqlKind.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlKind.java b/core/src/main/java/org/apache/calcite/sql/SqlKind.java
index 05a5eb5..a6da09d 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlKind.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlKind.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
+package org.apache.calcite.sql;
 
 import java.util.Collection;
 import java.util.EnumSet;
@@ -25,10 +25,10 @@ import java.util.Set;
  *
  * <p>The values are immutable, canonical constants, so you can use Kinds to
  * find particular types of expressions quickly. To identity a call to a common
- * operator such as '=', use {@link org.eigenbase.sql.SqlNode#isA}:</p>
+ * operator such as '=', use {@link org.apache.calcite.sql.SqlNode#isA}:</p>
  *
  * <blockquote>
- * <pre>exp.{@link org.eigenbase.sql.SqlNode#isA isA}({@link #EQUALS})</pre>
+ * exp.{@link org.apache.calcite.sql.SqlNode#isA isA}({@link #EQUALS})
  * </blockquote>
  *
  * <p>Only commonly-used nodes have their own type; other nodes are of type

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 eb1ce1e..de2b32a 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlLiteral.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlLiteral.java
@@ -14,21 +14,30 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
-
-import java.math.*;
-import java.nio.charset.*;
-import java.util.*;
-
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.fun.*;
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.sql.util.*;
-import org.eigenbase.sql.validate.*;
-import org.eigenbase.util.*;
-
-import static org.eigenbase.util.Static.RESOURCE;
+package org.apache.calcite.sql;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.sql.fun.SqlLiteralChainOperator;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.parser.SqlParserUtil;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.type.SqlTypeUtil;
+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.BitString;
+import org.apache.calcite.util.NlsString;
+import org.apache.calcite.util.Util;
+
+import java.math.BigDecimal;
+import java.nio.charset.Charset;
+import java.nio.charset.UnsupportedCharsetException;
+import java.util.Calendar;
+
+import static org.apache.calcite.util.Static.RESOURCE;
 
 /**
  * A <code>SqlLiteral</code> is a constant. It is, appropriately, immutable.
@@ -389,7 +398,7 @@ public class SqlLiteral extends SqlNode {
       return false;
     }
     SqlLiteral that = (SqlLiteral) obj;
-    return Util.equal(value, that.value);
+    return com.google.common.base.Objects.equal(value, that.value);
   }
 
   public int hashCode() {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/SqlMerge.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlMerge.java b/core/src/main/java/org/apache/calcite/sql/SqlMerge.java
index edd584b..59317e3 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlMerge.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlMerge.java
@@ -14,14 +14,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
+package org.apache.calcite.sql;
 
-import java.util.*;
+import org.apache.calcite.sql.parser.SqlParserPos;
+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.ImmutableNullableList;
+import org.apache.calcite.util.Pair;
 
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.sql.validate.*;
-import org.eigenbase.util.ImmutableNullableList;
-import org.eigenbase.util.Pair;
+import java.util.Iterator;
+import java.util.List;
 
 /**
  * A <code>SqlMerge</code> is a node of a parse tree which represents a MERGE
@@ -74,8 +77,7 @@ public class SqlMerge extends SqlCall {
         insertCall, sourceSelect, alias);
   }
 
-  @Override
-  public void setOperand(int i, SqlNode operand) {
+  @Override public void setOperand(int i, SqlNode operand) {
     switch (i) {
     case 0:
       targetTable = (SqlIdentifier) operand;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 2f357b3..75e3979 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlNode.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlNode.java
@@ -14,20 +14,27 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
+package org.apache.calcite.sql;
 
-import java.util.*;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.pretty.SqlPrettyWriter;
+import org.apache.calcite.sql.util.SqlString;
+import org.apache.calcite.sql.util.SqlVisitor;
+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.Util;
 
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.sql.pretty.*;
-import org.eigenbase.sql.util.*;
-import org.eigenbase.sql.validate.*;
-import org.eigenbase.util.*;
+import java.util.List;
+import java.util.Set;
 
 /**
- * A <code>SqlNode</code> is a SQL parse tree. It may be an {@link SqlOperator
- * operator}, {@link SqlLiteral literal}, {@link SqlIdentifier identifier}, and
- * so forth.
+ * A <code>SqlNode</code> is a SQL parse tree.
+ *
+ * <p>It may be an
+ * {@link SqlOperator operator}, {@link SqlLiteral literal},
+ * {@link SqlIdentifier identifier}, and so forth.
  */
 public abstract class SqlNode implements Cloneable {
   //~ Static fields/initializers ---------------------------------------------
@@ -71,8 +78,8 @@ public abstract class SqlNode implements Cloneable {
   }
 
   /**
-   * Returns the type of node this is, or {@link
-   * org.eigenbase.sql.SqlKind#OTHER} if it's nothing special.
+   * Returns the type of node this is, or
+   * {@link org.apache.calcite.sql.SqlKind#OTHER} if it's nothing special.
    *
    * @return a {@link SqlKind} value, never null
    * @see #isA
@@ -227,8 +234,8 @@ public abstract class SqlNode implements Cloneable {
    * Accepts a generic visitor.
    *
    * <p>Implementations of this method in subtypes simply call the appropriate
-   * <code>visit</code> method on the {@link org.eigenbase.sql.util.SqlVisitor
-   * visitor object}.
+   * <code>visit</code> method on the
+   * {@link org.apache.calcite.sql.util.SqlVisitor visitor object}.
    *
    * <p>The type parameter <code>R</code> must be consistent with the type
    * parameter of the visitor.
@@ -248,8 +255,8 @@ public abstract class SqlNode implements Cloneable {
   public abstract boolean equalsDeep(SqlNode node, boolean fail);
 
   /**
-   * Returns whether two nodes are equal (using {@link
-   * #equalsDeep(SqlNode, boolean)}) or are both null.
+   * Returns whether two nodes are equal (using
+   * {@link #equalsDeep(SqlNode, boolean)}) or are both null.
    *
    * @param node1 First expression
    * @param node2 Second expression
@@ -274,8 +281,8 @@ public abstract class SqlNode implements Cloneable {
    * This property is useful because it allows to safely aggregte infinite
    * streams of values.
    *
-   * <p>The default implementation returns {@link
-   * SqlMonotonicity#NOT_MONOTONIC}.
+   * <p>The default implementation returns
+   * {@link SqlMonotonicity#NOT_MONOTONIC}.
    */
   public SqlMonotonicity getMonotonicity(SqlValidatorScope scope) {
     return SqlMonotonicity.NOT_MONOTONIC;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 0c23fd0..4d77dac 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlNodeList.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlNodeList.java
@@ -14,18 +14,22 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
+package org.apache.calcite.sql;
 
-import java.util.*;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+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.eigenbase.sql.fun.*;
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.sql.util.*;
-import org.eigenbase.sql.validate.*;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
 
 /**
- * A <code>SqlNodeList</code> is a list of {@link SqlNode}s. It is also a {@link
- * SqlNode}, so may appear in a parse tree.
+ * A <code>SqlNodeList</code> is a list of {@link SqlNode}s. It is also a
+ * {@link SqlNode}, so may appear in a parse tree.
  */
 public class SqlNodeList extends SqlNode implements Iterable<SqlNode> {
   //~ Static fields/initializers ---------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/SqlNullSemantics.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlNullSemantics.java b/core/src/main/java/org/apache/calcite/sql/SqlNullSemantics.java
index 27bc330..314684f 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlNullSemantics.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlNullSemantics.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
+package org.apache.calcite.sql;
 
 /**
  * SqlNullSemantics defines the possible comparison rules for values which might

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/SqlNumericLiteral.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlNumericLiteral.java b/core/src/main/java/org/apache/calcite/sql/SqlNumericLiteral.java
index bf48850..b754a36 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlNumericLiteral.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlNumericLiteral.java
@@ -14,14 +14,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
+package org.apache.calcite.sql;
 
-import java.math.*;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.util.Util;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.util.*;
+import java.math.BigDecimal;
 
 /**
  * A numeric SQL literal.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/SqlOperandCountRange.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlOperandCountRange.java b/core/src/main/java/org/apache/calcite/sql/SqlOperandCountRange.java
index 66db8b5..038c263 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlOperandCountRange.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlOperandCountRange.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
+package org.apache.calcite.sql;
 
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 3cfbe20..9425514 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlOperator.java
@@ -14,25 +14,32 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
-
-import java.util.*;
-
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.sql.util.*;
-import org.eigenbase.sql.validate.*;
-import org.eigenbase.util.*;
-
-import static org.eigenbase.util.Static.RESOURCE;
+package org.apache.calcite.sql;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.SqlOperandTypeChecker;
+import org.apache.calcite.sql.type.SqlOperandTypeInference;
+import org.apache.calcite.sql.type.SqlReturnTypeInference;
+import org.apache.calcite.sql.util.SqlBasicVisitor;
+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.sql.validate.SqlValidatorUtil;
+import org.apache.calcite.util.Util;
+
+import java.util.List;
+
+import static org.apache.calcite.util.Static.RESOURCE;
 
 /**
  * A <code>SqlOperator</code> is a type of node in a SQL parse tree (it is NOT a
  * node in a SQL parse tree). It includes functions, operators such as '=', and
  * syntactic constructs such as 'case' statements. Operators may represent
  * query-level expressions (e.g. {@link SqlSelectOperator} or row-level
- * expressions (e.g. {@link org.eigenbase.sql.fun.SqlBetweenOperator}.
+ * expressions (e.g. {@link org.apache.calcite.sql.fun.SqlBetweenOperator}.
  *
  * <p>Operators have <em>formal operands</em>, meaning ordered (and optionally
  * named) placeholders for the values they operate on. For example, the division
@@ -299,8 +306,8 @@ public abstract class SqlOperator {
    * including parentheses if the operators on either side are of greater
    * precedence.
    *
-   * <p>The default implementation of this method delegates to {@link
-   * SqlSyntax#unparse}.
+   * <p>The default implementation of this method delegates to
+   * {@link SqlSyntax#unparse}.
    */
   public void unparse(
       SqlWriter writer,
@@ -356,8 +363,8 @@ public abstract class SqlOperator {
    * Validates a call to this operator.
    *
    * <p>This method should not perform type-derivation or perform validation
-   * related related to types. That is done later, by {@link
-   * #deriveType(SqlValidator, SqlValidatorScope, SqlCall)}. This method
+   * related related to types. That is done later, by
+   * {@link #deriveType(SqlValidator, SqlValidatorScope, SqlCall)}. This method
    * should focus on structural validation.
    *
    * <p>A typical implementation of this method first validates the operands,
@@ -439,8 +446,8 @@ public abstract class SqlOperator {
    * Subclasses must either override this method or supply an instance of
    * {@link SqlReturnTypeInference} to the constructor.
    *
-   * @param opBinding description of invocation (not necessarily a {@link
-   *                  SqlCall})
+   * @param opBinding description of invocation (not necessarily a
+   * {@link SqlCall})
    * @return inferred return type
    */
   public RelDataType inferReturnType(
@@ -665,9 +672,11 @@ public abstract class SqlOperator {
   }
 
   /**
-   * Accepts a {@link SqlVisitor}, directing an {@link
-   * org.eigenbase.sql.util.SqlBasicVisitor.ArgHandler} to visit operand of a
-   * call. The argument handler allows fine control about how the operands are
+   * Accepts a {@link SqlVisitor}, directing an
+   * {@link org.apache.calcite.sql.util.SqlBasicVisitor.ArgHandler}
+   * to visit an operand of a call.
+   *
+   * <p>The argument handler allows fine control about how the operands are
    * visited, and how the results are combined.
    *
    * @param visitor         Visitor

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/SqlOperatorBinding.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlOperatorBinding.java b/core/src/main/java/org/apache/calcite/sql/SqlOperatorBinding.java
index 16ef976..e767049 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlOperatorBinding.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlOperatorBinding.java
@@ -14,18 +14,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
+package org.apache.calcite.sql;
 
-import java.util.*;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.runtime.CalciteException;
+import org.apache.calcite.runtime.Resources;
+import org.apache.calcite.sql.validate.SqlValidatorException;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.resource.Resources;
-import org.eigenbase.sql.validate.*;
-import org.eigenbase.util.*;
+import java.util.AbstractList;
+import java.util.List;
 
 /**
- * <code>SqlOperatorBinding</code> represents the binding of an {@link
- * SqlOperator} to actual operands, along with any additional information
+ * <code>SqlOperatorBinding</code> represents the binding of an
+ * {@link SqlOperator} to actual operands, along with any additional information
  * required to validate those operands if needed.
  */
 public abstract class SqlOperatorBinding {
@@ -179,7 +181,7 @@ public abstract class SqlOperatorBinding {
    * @param e Validation error, not null
    * @return Error wrapped, if possible, with positional information
    */
-  public abstract EigenbaseException newError(
+  public abstract CalciteException newError(
       Resources.ExInst<SqlValidatorException> e);
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/SqlOperatorTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlOperatorTable.java b/core/src/main/java/org/apache/calcite/sql/SqlOperatorTable.java
index e23a6de..adfe6fa 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlOperatorTable.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlOperatorTable.java
@@ -14,9 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
+package org.apache.calcite.sql;
 
-import java.util.*;
+import java.util.List;
 
 /**
  * SqlOperatorTable defines a directory interface for enumerating and looking up

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/SqlOrderBy.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlOrderBy.java b/core/src/main/java/org/apache/calcite/sql/SqlOrderBy.java
index aa72e2c..421b9e2 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlOrderBy.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlOrderBy.java
@@ -14,25 +14,24 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
+package org.apache.calcite.sql;
 
-import java.util.List;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.util.ImmutableNullableList;
 
-import org.eigenbase.sql.parser.SqlParserPos;
-import org.eigenbase.util.ImmutableNullableList;
+import java.util.List;
 
 /**
  * Parse tree node that represents an {@code ORDER BY} on a query other than a
  * {@code SELECT} (e.g. {@code VALUES} or {@code UNION}).
  *
  * <p>It is a purely syntactic operator, and is eliminated by
- * {@link org.eigenbase.sql.validate.SqlValidatorImpl#performUnconditionalRewrites}
+ * {@link org.apache.calcite.sql.validate.SqlValidatorImpl#performUnconditionalRewrites}
  * and replaced with the ORDER_OPERAND of SqlSelect.</p>
  */
 public class SqlOrderBy extends SqlCall {
   public static final SqlSpecialOperator OPERATOR = new Operator() {
-    @Override
-    public SqlCall createCall(SqlLiteral functionQualifier,
+    @Override public SqlCall createCall(SqlLiteral functionQualifier,
         SqlParserPos pos, SqlNode... operands) {
       return new SqlOrderBy(pos, operands[0], (SqlNodeList) operands[1],
           operands[2], operands[3]);
@@ -121,4 +120,4 @@ public class SqlOrderBy extends SqlCall {
   }
 }
 
-// End SqlOrderByOperator.java
+// End SqlOrderBy.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/SqlOverOperator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlOverOperator.java b/core/src/main/java/org/apache/calcite/sql/SqlOverOperator.java
index bc70306..6255aaa 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlOverOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlOverOperator.java
@@ -14,16 +14,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
+package org.apache.calcite.sql;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.sql.util.*;
-import org.eigenbase.sql.validate.*;
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.ReturnTypes;
+import org.apache.calcite.sql.util.SqlBasicVisitor;
+import org.apache.calcite.sql.util.SqlVisitor;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql.validate.SqlValidatorScope;
 
-import net.hydromatic.linq4j.Ord;
-
-import static org.eigenbase.util.Static.RESOURCE;
+import static org.apache.calcite.util.Static.RESOURCE;
 
 /**
  * An operator describing a window function specification.
@@ -31,9 +33,11 @@ import static org.eigenbase.util.Static.RESOURCE;
  * <p>Operands are as follows:</p>
  *
  * <ul>
- * <li>0: name of window function ({@link org.eigenbase.sql.SqlCall})</li>
- * <li>1: window name ({@link org.eigenbase.sql.SqlLiteral}) or window in-line
- * specification ({@link SqlWindow})</li>
+ * <li>0: name of window function ({@link org.apache.calcite.sql.SqlCall})</li>
+ *
+ * <li>1: window name ({@link org.apache.calcite.sql.SqlLiteral}) or
+ * window in-line specification ({@link SqlWindow})</li>
+ *
  * </ul>
  */
 public class SqlOverOperator extends SqlBinaryOperator {
@@ -93,8 +97,7 @@ public class SqlOverOperator extends SqlBinaryOperator {
     final SqlCall aggCall = (SqlCall) agg;
 
     SqlCallBinding opBinding = new SqlCallBinding(validator, scope, aggCall) {
-      @Override
-      public int getGroupCount() {
+      @Override public int getGroupCount() {
         return groupCount;
       }
     };

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 a8bbad0..9b36a08 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlPostfixOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlPostfixOperator.java
@@ -14,12 +14,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
+package org.apache.calcite.sql;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.sql.validate.*;
-import org.eigenbase.util.*;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.type.SqlOperandTypeChecker;
+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.Util;
 
 /**
  * A postfix unary operator.
@@ -85,8 +88,7 @@ public class SqlPostfixOperator extends SqlOperator {
     return type;
   }
 
-  @Override
-  public boolean validRexOperands(int count, boolean fail) {
+  @Override public boolean validRexOperands(int count, boolean fail) {
     if (count != 1) {
       assert !fail : "wrong operand count " + count + " for " + this;
       return false;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 6dd591c..b99abf2 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlPrefixOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlPrefixOperator.java
@@ -14,12 +14,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
+package org.apache.calcite.sql;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.sql.validate.*;
-import org.eigenbase.util.*;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.type.SqlOperandTypeChecker;
+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.SqlMonotonicity;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql.validate.SqlValidatorScope;
+import org.apache.calcite.util.Util;
 
 /**
  * A unary operator.
@@ -95,8 +100,7 @@ public class SqlPrefixOperator extends SqlOperator {
     return super.getMonotonicity(call, scope);
   }
 
-  @Override
-  public boolean validRexOperands(int count, boolean fail) {
+  @Override public boolean validRexOperands(int count, boolean fail) {
     if (count != 1) {
       assert !fail : "wrong operand count " + count + " for " + this;
       return false;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/SqlProcedureCallOperator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlProcedureCallOperator.java b/core/src/main/java/org/apache/calcite/sql/SqlProcedureCallOperator.java
index efae473..ef02914 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlProcedureCallOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlProcedureCallOperator.java
@@ -14,13 +14,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
+package org.apache.calcite.sql;
 
-import java.util.*;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.validate.SqlValidator;
 
-import org.eigenbase.sql.fun.*;
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.sql.validate.*;
+import java.util.Collections;
 
 /**
  * SqlProcedureCallOperator represents the CALL statement. It takes a single

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/SqlRankFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlRankFunction.java b/core/src/main/java/org/apache/calcite/sql/SqlRankFunction.java
index 5c71816..e157671 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlRankFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlRankFunction.java
@@ -14,18 +14,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
+package org.apache.calcite.sql;
 
-import java.util.List;
-
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.sql.validate.*;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.ReturnTypes;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql.validate.SqlValidatorScope;
 
 import com.google.common.collect.ImmutableList;
 
-import static org.eigenbase.util.Static.RESOURCE;
+import java.util.List;
+
+import static org.apache.calcite.util.Static.RESOURCE;
 
 /**
  * Operator which aggregates sets of values into a result.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/SqlSampleSpec.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlSampleSpec.java b/core/src/main/java/org/apache/calcite/sql/SqlSampleSpec.java
index 0c0a90f..498b9ed 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlSampleSpec.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlSampleSpec.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
+package org.apache.calcite.sql;
 
 /**
  * Specification of a SQL sample.
@@ -29,8 +29,8 @@ package org.eigenbase.sql;
  * <p>declares a sample which is created using {@link #createNamed}.</p>
  *
  * <p>A sample is not a {@link SqlNode}. To include it in a parse tree, wrap it
- * as a literal, viz: {@link SqlLiteral#createSample(SqlSampleSpec,
- * SqlParserPos)}.
+ * as a literal, viz:
+ * {@link SqlLiteral#createSample(SqlSampleSpec, org.apache.calcite.sql.parser.SqlParserPos)}.
  */
 public abstract class SqlSampleSpec {
   //~ Constructors -----------------------------------------------------------
@@ -80,6 +80,7 @@ public abstract class SqlSampleSpec {
 
   //~ Inner Classes ----------------------------------------------------------
 
+  /** Sample specification that orders substitution. */
   public static class SqlSubstitutionSampleSpec extends SqlSampleSpec {
     private final String name;
 
@@ -93,11 +94,12 @@ public abstract class SqlSampleSpec {
 
     public String toString() {
       return "SUBSTITUTE("
-          + SqlDialect.EIGENBASE.quoteStringLiteral(name)
+          + SqlDialect.CALCITE.quoteStringLiteral(name)
           + ")";
     }
   }
 
+  /** Sample specification. */
   public static class SqlTableSampleSpec extends SqlSampleSpec {
     private final boolean isBernoulli;
     private final float samplePercentage;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/SqlSelect.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlSelect.java b/core/src/main/java/org/apache/calcite/sql/SqlSelect.java
index dba0ce6..2fa09e5 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlSelect.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlSelect.java
@@ -14,13 +14,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
+package org.apache.calcite.sql;
 
-import java.util.List;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql.validate.SqlValidatorScope;
+import org.apache.calcite.util.ImmutableNullableList;
 
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.sql.validate.*;
-import org.eigenbase.util.ImmutableNullableList;
+import java.util.List;
 
 /**
  * A <code>SqlSelect</code> is a node of a parse tree which represents a select
@@ -87,8 +88,7 @@ public class SqlSelect extends SqlCall {
         groupBy, having, windowDecls, orderBy, offset, fetch);
   }
 
-  @Override
-  public void setOperand(int i, SqlNode operand) {
+  @Override public void setOperand(int i, SqlNode operand) {
     switch (i) {
     case 0:
       keywordList = (SqlNodeList) operand;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/SqlSelectKeyword.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlSelectKeyword.java b/core/src/main/java/org/apache/calcite/sql/SqlSelectKeyword.java
index adff68a..c016289 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlSelectKeyword.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlSelectKeyword.java
@@ -14,9 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
+package org.apache.calcite.sql;
 
-import org.eigenbase.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.parser.SqlParserPos;
 
 /**
  * Defines the keywords which can occur immediately after the "SELECT" keyword.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/SqlSelectOperator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlSelectOperator.java b/core/src/main/java/org/apache/calcite/sql/SqlSelectOperator.java
index 6f58867..858e58f 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlSelectOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlSelectOperator.java
@@ -14,13 +14,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
+package org.apache.calcite.sql;
 
-import java.util.*;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.ReturnTypes;
+import org.apache.calcite.sql.util.SqlBasicVisitor;
+import org.apache.calcite.sql.util.SqlVisitor;
 
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.sql.util.*;
+import java.util.ArrayList;
 
 /**
  * An operator describing a query. (Not a query itself.)
@@ -87,8 +88,8 @@ public class SqlSelectOperator extends SqlOperator {
    *                    returning first row
    * @param fetch       Expression for number of rows to fetch
    * @param pos         The parser position, or
-   *                    {@link org.eigenbase.sql.parser.SqlParserPos#ZERO} if not
-   *                    specified; must not be null.
+   *                    {@link org.apache.calcite.sql.parser.SqlParserPos#ZERO}
+   *                    if not specified; must not be null.
    * @return A {@link SqlSelect}, never null
    */
   public SqlSelect createCall(

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/SqlSetOperator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlSetOperator.java b/core/src/main/java/org/apache/calcite/sql/SqlSetOperator.java
index 29d3417..d4699f3 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlSetOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlSetOperator.java
@@ -14,10 +14,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
+package org.apache.calcite.sql;
 
-import org.eigenbase.sql.type.*;
-import org.eigenbase.sql.validate.*;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.ReturnTypes;
+import org.apache.calcite.sql.type.SqlOperandTypeChecker;
+import org.apache.calcite.sql.type.SqlOperandTypeInference;
+import org.apache.calcite.sql.type.SqlReturnTypeInference;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql.validate.SqlValidatorScope;
 
 /**
  * SqlSetOperator represents a relational set theory operator (UNION, INTERSECT,

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/SqlSetOption.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlSetOption.java b/core/src/main/java/org/apache/calcite/sql/SqlSetOption.java
index 6b33e11..5f6004f 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlSetOption.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlSetOption.java
@@ -14,16 +14,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
+package org.apache.calcite.sql;
 
-import java.util.List;
-
-import org.eigenbase.sql.parser.SqlParserPos;
-import org.eigenbase.sql.validate.SqlValidator;
-import org.eigenbase.sql.validate.SqlValidatorScope;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql.validate.SqlValidatorScope;
 
 import com.google.common.collect.ImmutableList;
 
+import java.util.List;
+
 /**
  * SQL parse tree node to represent <code>ALTER scope SET option = value</code>
  * statement.
@@ -41,9 +41,10 @@ public class SqlSetOption extends SqlCall {
 
   String name;
 
-  /** Value of the option. May be a {@link org.eigenbase.sql.SqlLiteral} or
-   * a {@link org.eigenbase.sql.SqlIdentifier} with one part. Reserved words
-   * (currently just 'ON') are converted to identifiers by the parser. */
+  /** Value of the option. May be a {@link org.apache.calcite.sql.SqlLiteral} or
+   * a {@link org.apache.calcite.sql.SqlIdentifier} with one
+   * part. Reserved words (currently just 'ON') are converted to
+   * identifiers by the parser. */
   SqlNode value;
 
   /**
@@ -65,8 +66,7 @@ public class SqlSetOption extends SqlCall {
     assert value != null;
   }
 
-  @Override
-  public SqlKind getKind() {
+  @Override public SqlKind getKind() {
     return SqlKind.SET_OPTION;
   }
 
@@ -74,8 +74,7 @@ public class SqlSetOption extends SqlCall {
     return OPERATOR;
   }
 
-  @Override
-  public List<SqlNode> getOperandList() {
+  @Override public List<SqlNode> getOperandList() {
     return ImmutableList.of(
         new SqlIdentifier(scope, SqlParserPos.ZERO),
         new SqlIdentifier(name, SqlParserPos.ZERO),

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/SqlSpecialOperator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlSpecialOperator.java b/core/src/main/java/org/apache/calcite/sql/SqlSpecialOperator.java
index 03b1f54..efecf9d 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlSpecialOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlSpecialOperator.java
@@ -14,12 +14,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
+package org.apache.calcite.sql;
 
-import java.util.*;
+import org.apache.calcite.sql.type.SqlOperandTypeChecker;
+import org.apache.calcite.sql.type.SqlOperandTypeInference;
+import org.apache.calcite.sql.type.SqlReturnTypeInference;
+import org.apache.calcite.util.Util;
 
-import org.eigenbase.sql.type.*;
-import org.eigenbase.util.*;
+import java.util.List;
 
 /**
  * Generic operator for nodes with special syntax.
@@ -78,14 +80,14 @@ public class SqlSpecialOperator extends SqlOperator {
    * precedence and associativity. Returns the ordinal of the node which
    * replaced the expression.
    *
-   * <p>The default implementation throws {@link
-   * UnsupportedOperationException}.
+   * <p>The default implementation throws
+   * {@link UnsupportedOperationException}.
    *
    * @param ordinal indicating the ordinal of the current operator in the list
    *                on which a possible reduction can be made
-   * @param list    List of alternating {@link
-   *                org.eigenbase.sql.parser.SqlParserUtil.ToTreeListItem} and {@link
-   *                SqlNode}
+   * @param list    List of alternating
+   *     {@link org.apache.calcite.sql.parser.SqlParserUtil.ToTreeListItem} and
+   *     {@link SqlNode}
    * @return ordinal of the node which replaced the expression
    */
   public int reduceExpr(

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/SqlStateCodes.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlStateCodes.java b/core/src/main/java/org/apache/calcite/sql/SqlStateCodes.java
index c048200..5753f47 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlStateCodes.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlStateCodes.java
@@ -14,11 +14,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
+package org.apache.calcite.sql;
 
 /**
- * Contains {@link org.eigenbase.util.Glossary#SQL2003} SQL state codes. Sql
- * Sate codes are defined in
+ * Contains {@link org.apache.calcite.util.Glossary#SQL2003} SQL state codes.
+ *
+ * <p>SQL State codes are defined in
  *
  * <pre><code> &#64;sql.2003 Part 2 Section 23.1</code></pre>
  */

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 a198150..f3e9bbc 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlSyntax.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlSyntax.java
@@ -14,9 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
+package org.apache.calcite.sql;
 
-import org.eigenbase.util.*;
+import org.apache.calcite.util.Util;
 
 /**
  * Enumeration of possible syntactic types of {@link SqlOperator operators}.
@@ -141,8 +141,7 @@ public enum SqlSyntax {
         SqlCall call,
         int leftPrec,
         int rightPrec) {
-      throw Util.newInternal(
-          "Internal operator '" + operator
+      throw Util.newInternal("Internal operator '" + operator
           + "' cannot be un-parsed");
     }
   };

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/SqlTimeLiteral.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlTimeLiteral.java b/core/src/main/java/org/apache/calcite/sql/SqlTimeLiteral.java
index 2f3ad18..f80990e 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlTimeLiteral.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlTimeLiteral.java
@@ -14,13 +14,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
+package org.apache.calcite.sql;
 
-import java.util.*;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.util.DateTimeUtil;
 
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.util14.DateTimeUtil;
+import java.util.Calendar;
 
 /**
  * A SQL literal representing a TIME value, for example <code>TIME

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/SqlTimestampLiteral.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlTimestampLiteral.java b/core/src/main/java/org/apache/calcite/sql/SqlTimestampLiteral.java
index cbd9377..f5db1d8 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlTimestampLiteral.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlTimestampLiteral.java
@@ -14,13 +14,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
+package org.apache.calcite.sql;
 
-import java.util.*;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.util.DateTimeUtil;
 
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.util14.DateTimeUtil;
+import java.util.Calendar;
 
 /**
  * A SQL literal representing a TIMESTAMP value, for example <code>TIMESTAMP

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/SqlUnnestOperator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlUnnestOperator.java b/core/src/main/java/org/apache/calcite/sql/SqlUnnestOperator.java
index c8106c5..54cbb99 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlUnnestOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlUnnestOperator.java
@@ -14,10 +14,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
+package org.apache.calcite.sql;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.type.*;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.type.ArraySqlType;
+import org.apache.calcite.sql.type.MultisetSqlType;
+import org.apache.calcite.sql.type.OperandTypes;
 
 /**
  * The <code>UNNEST</code> operator.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/SqlUnresolvedFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlUnresolvedFunction.java b/core/src/main/java/org/apache/calcite/sql/SqlUnresolvedFunction.java
index 5fdbe78..d41b4e3 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlUnresolvedFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlUnresolvedFunction.java
@@ -14,15 +14,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
+package org.apache.calcite.sql;
 
-import java.util.List;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.type.SqlOperandTypeChecker;
+import org.apache.calcite.sql.type.SqlOperandTypeInference;
+import org.apache.calcite.sql.type.SqlReturnTypeInference;
+import org.apache.calcite.sql.type.SqlTypeName;
 
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.sql.type.SqlOperandTypeChecker;
-import org.eigenbase.sql.type.SqlOperandTypeInference;
-import org.eigenbase.sql.type.SqlReturnTypeInference;
-import org.eigenbase.sql.type.SqlTypeName;
+import java.util.List;
 
 /**
  * Placeholder for an unresolved function.
@@ -62,8 +62,7 @@ public class SqlUnresolvedFunction extends SqlFunction {
    * functions. Return the return type to be 'ANY', so we don't
    * fail.
    */
-  @Override
-  public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
+  @Override public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
     return opBinding.getTypeFactory().createSqlType(SqlTypeName.ANY);
   }
 }


[56/58] [abbrv] incubator-calcite git commit: [CALCITE-370] Support GROUPING SETS, CUBE, ROLLUP in SQL and algebra

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6d79b5eb/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
----------------------------------------------------------------------
diff --git a/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml b/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
index 1ada2df..f7036ab 100644
--- a/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
+++ b/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
@@ -934,7 +934,7 @@ Sort(sort0=[$1], dir0=[ASC])
     </TestCase>
     <TestCase name="testOrderByOrdinalDesc">
         <Resource name="sql">
-            <![CDATA[select empno + 1, deptno, empno from emp order by 2 desc]]>
+            <![CDATA[select empno + 1, deptno, empno from emp order by 2.5 desc]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
@@ -1148,9 +1148,9 @@ LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$
     </TestCase>
     <TestCase name="testOverOrderWindow">
         <Resource name="sql">
-            <![CDATA[select last_value(deptno) over w
+            <![CDATA[select last_value(deptno) over (order by empno)
 from emp
-window w as (order by empno)]]>
+]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
@@ -1161,9 +1161,9 @@ LogicalProject(EXPR$0=[LAST_VALUE($7) OVER (ORDER BY $0 RANGE BETWEEN UNBOUNDED
     </TestCase>
     <TestCase name="testOverOrderFollowingWindow">
         <Resource name="sql">
-            <![CDATA[select last_value(deptno) over w
+            <![CDATA[select last_value(deptno) over (order by empno rows 2 following)
 from emp
-window w as (order by empno rows 2 following)]]>
+]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
@@ -1977,4 +1977,155 @@ LogicalAggregate(group=[{0}], EXPR$1=[MAX($1)], EXPR$2=[MAX($2)])
 ]]>
         </Resource>
     </TestCase>
+    <TestCase name="testSingletonGroupingSet">
+        <Resource name="sql">
+            <![CDATA[select sum(sal) from emp group by grouping sets (deptno)]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(EXPR$0=[$1])
+  LogicalAggregate(group=[{0}], EXPR$0=[SUM($1)])
+    LogicalProject(DEPTNO=[$7], SAL=[$5])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testGroupEmpty">
+        <Resource name="sql">
+            <![CDATA[select sum(deptno) from emp group by ()]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalAggregate(group=[{}], EXPR$0=[SUM($0)])
+  LogicalProject(DEPTNO=[$7])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testGroupByWithDuplicates">
+        <Resource name="sql">
+            <![CDATA[select sum(sal) from emp group by (), ()]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalAggregate(group=[{}], EXPR$0=[SUM($0)])
+  LogicalProject(SAL=[$5])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testGroupingSets">
+        <Resource name="sql">
+            <![CDATA[select deptno, ename, sum(sal) from emp
+group by grouping sets ((deptno), (ename, deptno))
+order by 2]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+Sort(sort0=[$1], dir0=[ASC])
+  LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}]], EXPR$2=[SUM($2)])
+    LogicalProject(DEPTNO=[$7], ENAME=[$1], SAL=[$5])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testDuplicateGroupingSets">
+        <Resource name="sql">
+            <![CDATA[select sum(sal) from emp
+group by sal,
+  grouping sets (deptno,
+    grouping sets ((deptno, ename), ename),
+      (ename)),
+  ()]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(EXPR$0=[$3])
+  LogicalAggregate(group=[{0, 1, 2}], groups=[[{0, 1, 2}, {0, 1}, {0, 2}]], EXPR$0=[SUM($0)])
+    LogicalProject(SAL=[$5], DEPTNO=[$7], ENAME=[$1])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testGroupingSetsCartesianProduct">
+        <Resource name="sql">
+            <![CDATA[select 1 from (values (1, 2, 3, 4)) as t(a, b, c, d)
+group by grouping sets (a, b), grouping sets (c, d)]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(EXPR$0=[1])
+  LogicalAggregate(group=[{0, 1, 2, 3}], groups=[[{0, 2}, {0, 3}, {1, 2}, {1, 3}]])
+    LogicalValues(tuples=[[{ 1, 2, 3, 4 }]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testGroupingSetsCartesianProduct2">
+        <Resource name="sql">
+            <![CDATA[select 1 from (values (1, 2, 3, 4)) as t(a, b, c, d)
+group by grouping sets (a, (a, b)), grouping sets (c), d]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(EXPR$0=[1])
+  LogicalAggregate(group=[{0, 1, 2, 3}], groups=[[{0, 1, 2, 3}, {0, 2, 3}]])
+    LogicalValues(tuples=[[{ 1, 2, 3, 4 }]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testRollup">
+        <Resource name="sql">
+            <![CDATA[select 1 from (values (1, 2, 3, 4)) as t(a, b, c, d)
+group by rollup(a, b), rollup(c, d)]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(EXPR$0=[1])
+  LogicalAggregate(group=[{0, 1, 2, 3}], groups=[[{0, 1, 2, 3}, {0, 1, 2}, {0, 1}, {0, 2, 3}, {0, 2}, {0}, {2, 3}, {2}, {}]])
+    LogicalValues(tuples=[[{ 1, 2, 3, 4 }]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testCube">
+        <Resource name="sql">
+            <![CDATA[select 1 from (values (1, 2, 3, 4)) as t(a, b, c, d)
+group by cube(a, b)]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(EXPR$0=[1])
+  LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {1}, {}]])
+    LogicalProject(EXPR$0=[$0], EXPR$1=[$1])
+      LogicalValues(tuples=[[{ 1, 2, 3, 4 }]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testRollupTuples">
+        <Resource name="sql">
+            <![CDATA[select 1 from (values (1, 2, 3, 4)) as t(a, b, c, d)
+group by rollup(b, (a, d))]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(EXPR$0=[1])
+  LogicalAggregate(group=[{0, 1, 2}], groups=[[{0, 1, 2}, {0}, {}]])
+    LogicalProject(EXPR$1=[$1], EXPR$0=[$0], EXPR$3=[$3])
+      LogicalValues(tuples=[[{ 1, 2, 3, 4 }]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testGroupingSetsWith">
+        <Resource name="sql">
+            <![CDATA[with t(a, b, c, d) as (values (1, 2, 3, 4))
+select 1 from t
+group by rollup(a, b), rollup(c, d)]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(EXPR$0=[1])
+  LogicalAggregate(group=[{0, 1, 2, 3}], groups=[[{0, 1, 2, 3}, {0, 1, 2}, {0, 1}, {0, 2, 3}, {0, 2}, {0}, {2, 3}, {2}, {}]])
+    LogicalValues(tuples=[[{ 1, 2, 3, 4 }]])
+]]>
+        </Resource>
+    </TestCase>
 </Root>

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6d79b5eb/core/src/test/resources/sql/agg.oq
----------------------------------------------------------------------
diff --git a/core/src/test/resources/sql/agg.oq b/core/src/test/resources/sql/agg.oq
index 51bed8b..5c1adb2 100644
--- a/core/src/test/resources/sql/agg.oq
+++ b/core/src/test/resources/sql/agg.oq
@@ -148,4 +148,125 @@ select count(mod(deptno, 20), gender) as c from emps;
 
 !ok
 
+# Basic GROUPING SETS
+select deptno, count(*) as c from emps group by grouping sets ((), (deptno));
++--------+---+
+| DEPTNO | C |
++--------+---+
+|      0 | 5 |
+|     10 | 1 |
+|     20 | 2 |
+|     40 | 2 |
++--------+---+
+(4 rows)
+
+!ok
+
+# GROUPING SETS on expression
+select deptno + 1, count(*) as c from emps group by grouping sets ((), (deptno + 1));
++--------+---+
+| EXPR$0 | C |
++--------+---+
+|      0 | 5 |
+|     11 | 1 |
+|     21 | 2 |
+|     41 | 2 |
++--------+---+
+(4 rows)
+
+!ok
+
+# CUBE
+!if (false) {
+select deptno + 1, count(*) as c from emps group by cube(deptno, gender);
+!ok
+!}
+
+# ROLLUP on 1 column
+select deptno + 1, count(*) as c from emps group by rollup(deptno);
++--------+---+
+| EXPR$0 | C |
++--------+---+
+|      1 | 5 |
+|     11 | 1 |
+|     21 | 2 |
+|     41 | 2 |
++--------+---+
+(4 rows)
+
+!ok
+
+# ROLLUP on 2 columns; project columns in different order
+select gender, deptno + 1, count(*) as c
+from emps
+group by rollup(deptno, gender);
++--------+--------+---+
+| GENDER | EXPR$1 | C |
++--------+--------+---+
+| F      |     21 | 1 |
+| F      |     41 | 1 |
+| M      |     21 | 1 |
+| M      |     41 | 1 |
+|        |      1 | 5 |
+|        |     11 | 1 |
+|        |     11 | 1 |
+|        |     21 | 2 |
+|        |     41 | 2 |
++--------+--------+---+
+(9 rows)
+
+!ok
+
+# ROLLUP on column with nulls
+# Note the two rows with NULL key (one represents ALL)
+select gender, count(*) as c
+from emps
+group by rollup(gender);
++--------+---+
+| GENDER | C |
++--------+---+
+| F      | 2 |
+| M      | 2 |
+|        | 1 |
+|        | 5 |
++--------+---+
+(4 rows)
+
+!ok
+
+# ROLLUP plus ORDER BY
+select gender, count(*) as c
+from emps
+group by rollup(gender)
+order by c desc;
++--------+---+
+| GENDER | C |
++--------+---+
+|        | 5 |
+| F      | 2 |
+| M      | 2 |
+|        | 1 |
++--------+---+
+(4 rows)
+
+!ok
+
+# ROLLUP cartesian product
+!if (false) {
+select deptno + 1, count(*) as c from emps group by rollup(deptno, gender), rollup(city);
+!ok
+!}
+
+# ROLLUP with HAVING
+select deptno + 1, count(*) as c from emps group by rollup(deptno)
+having count(*) > 3;
++--------+---+
+| EXPR$0 | C |
++--------+---+
+|      1 | 5 |
++--------+---+
+(1 row)
+
+!ok
+
 # End agg.oq

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6d79b5eb/linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java b/linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
index 0e2d436..b2f64a7 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
@@ -666,6 +666,30 @@ public abstract class EnumerableDefaults {
   }
 
   /**
+   * Groups the elements of a sequence according to a list of
+   * specified key selector functions, initializing an accumulator for each
+   * group and adding to it each time an element with the same key is seen.
+   * Creates a result value from each accumulator and its key using a
+   * specified function.
+   *
+   * <p>This method exists to support SQL {@code GROUPING SETS}.
+   * It does not correspond to any method in {@link Enumerable}.
+   */
+  public static <TSource, TKey, TAccumulate, TResult> Enumerable<TResult>
+  groupByMultiple(Enumerable<TSource> enumerable,
+      List<Function1<TSource, TKey>> keySelectors,
+      Function0<TAccumulate> accumulatorInitializer,
+      Function2<TAccumulate, TSource, TAccumulate> accumulatorAdder,
+      final Function2<TKey, TAccumulate, TResult> resultSelector) {
+    return groupByMultiple_(new HashMap<TKey, TAccumulate>(),
+        enumerable,
+        keySelectors,
+        accumulatorInitializer,
+        accumulatorAdder,
+        resultSelector);
+  }
+
+  /**
    * Groups the elements of a sequence according to a
    * specified key selector function, initializing an accumulator for each
    * group and adding to it each time an element with the same key is seen.
@@ -710,26 +734,42 @@ public abstract class EnumerableDefaults {
     } finally {
       os.close();
     }
-    return new AbstractEnumerable2<TResult>() {
-      public Iterator<TResult> iterator() {
-        final Iterator<Map.Entry<TKey, TAccumulate>> iterator =
-            map.entrySet().iterator();
-        return new Iterator<TResult>() {
-          public boolean hasNext() {
-            return iterator.hasNext();
-          }
-
-          public TResult next() {
-            final Map.Entry<TKey, TAccumulate> entry = iterator.next();
-            return resultSelector.apply(entry.getKey(), entry.getValue());
-          }
+    return new LookupResultEnumerable<TResult, TKey, TAccumulate>(map,
+        resultSelector);
+  }
 
-          public void remove() {
-            throw new UnsupportedOperationException();
+  private static <TSource, TKey, TAccumulate, TResult> Enumerable<TResult>
+  groupByMultiple_(final Map<TKey, TAccumulate> map,
+      Enumerable<TSource> enumerable,
+      List<Function1<TSource, TKey>> keySelectors,
+      Function0<TAccumulate> accumulatorInitializer,
+      Function2<TAccumulate, TSource, TAccumulate> accumulatorAdder,
+      final Function2<TKey, TAccumulate, TResult> resultSelector) {
+    final Enumerator<TSource> os = enumerable.enumerator();
+    try {
+      while (os.moveNext()) {
+        for (Function1<TSource, TKey> keySelector : keySelectors) {
+          TSource o = os.current();
+          TKey key = keySelector.apply(o);
+          TAccumulate accumulator = map.get(key);
+          if (accumulator == null) {
+            accumulator = accumulatorInitializer.apply();
+            accumulator = accumulatorAdder.apply(accumulator, o);
+            map.put(key, accumulator);
+          } else {
+            TAccumulate accumulator0 = accumulator;
+            accumulator = accumulatorAdder.apply(accumulator, o);
+            if (accumulator != accumulator0) {
+              map.put(key, accumulator);
+            }
           }
-        };
+        }
       }
-    };
+    } finally {
+      os.close();
+    }
+    return new LookupResultEnumerable<TResult, TKey, TAccumulate>(map,
+        resultSelector);
   }
 
   private static <TSource, TKey, TResult> Enumerable<TResult>
@@ -2364,6 +2404,38 @@ public abstract class EnumerableDefaults {
       return map.values();
     }
   }
+
+  /** Reads a populated map, applying a selector function. */
+  private static class LookupResultEnumerable<TResult, TKey, TAccumulate>
+      extends AbstractEnumerable2<TResult> {
+    private final Map<TKey, TAccumulate> map;
+    private final Function2<TKey, TAccumulate, TResult> resultSelector;
+
+    public LookupResultEnumerable(Map<TKey, TAccumulate> map,
+        Function2<TKey, TAccumulate, TResult> resultSelector) {
+      this.map = map;
+      this.resultSelector = resultSelector;
+    }
+
+    public Iterator<TResult> iterator() {
+      final Iterator<Map.Entry<TKey, TAccumulate>> iterator =
+          map.entrySet().iterator();
+      return new Iterator<TResult>() {
+        public boolean hasNext() {
+          return iterator.hasNext();
+        }
+
+        public TResult next() {
+          final Map.Entry<TKey, TAccumulate> entry = iterator.next();
+          return resultSelector.apply(entry.getKey(), entry.getValue());
+        }
+
+        public void remove() {
+          throw new UnsupportedOperationException();
+        }
+      };
+    }
+  }
 }
 
 // End EnumerableDefaults.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6d79b5eb/linq4j/src/main/java/org/apache/calcite/linq4j/Linq4j.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/Linq4j.java b/linq4j/src/main/java/org/apache/calcite/linq4j/Linq4j.java
index 2b11705..b50998d 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/Linq4j.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/Linq4j.java
@@ -16,6 +16,8 @@
  */
 package org.apache.calcite.linq4j;
 
+import com.google.common.collect.Lists;
+
 import java.io.Closeable;
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
@@ -375,6 +377,21 @@ public abstract class Linq4j {
     return new CartesianProductEnumerator<T>(enumerators);
   }
 
+  /** Returns the cartesian product of an iterable of iterables. */
+  public static <T> Iterable<List<T>> product(
+      final Iterable<? extends Iterable<T>> iterables) {
+    return new Iterable<List<T>>() {
+      public Iterator<List<T>> iterator() {
+        final List<Enumerator<T>> enumerators = Lists.newArrayList();
+        for (Iterable<T> iterable : iterables) {
+          enumerators.add(iterableEnumerator(iterable));
+        }
+        return enumeratorIterator(
+            new CartesianProductEnumerator<T>(enumerators));
+      }
+    };
+  }
+
   /**
    * Returns whether the arguments are equal to each other.
    *

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6d79b5eb/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoAggregate.java
----------------------------------------------------------------------
diff --git a/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoAggregate.java b/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoAggregate.java
index 81940f7..35f4661 100644
--- a/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoAggregate.java
+++ b/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoAggregate.java
@@ -45,10 +45,12 @@ public class MongoAggregate
       RelOptCluster cluster,
       RelTraitSet traitSet,
       RelNode child,
+      boolean indicator,
       ImmutableBitSet groupSet,
+      List<ImmutableBitSet> groupSets,
       List<AggregateCall> aggCalls)
       throws InvalidRelException {
-    super(cluster, traitSet, child, groupSet, aggCalls);
+    super(cluster, traitSet, child, indicator, groupSet, groupSets, aggCalls);
     assert getConvention() == MongoRel.CONVENTION;
     assert getConvention() == child.getConvention();
 
@@ -58,13 +60,21 @@ public class MongoAggregate
             "distinct aggregation not supported");
       }
     }
+    switch (getGroupType()) {
+    case SIMPLE:
+      break;
+    default:
+      throw new InvalidRelException("unsupported group type: "
+          + getGroupType());
+    }
   }
 
   @Override public Aggregate copy(RelTraitSet traitSet, RelNode input,
-      ImmutableBitSet groupSet, List<AggregateCall> aggCalls) {
+      boolean indicator, ImmutableBitSet groupSet,
+      List<ImmutableBitSet> groupSets, List<AggregateCall> aggCalls) {
     try {
-      return new MongoAggregate(getCluster(), traitSet, input, groupSet,
-          aggCalls);
+      return new MongoAggregate(getCluster(), traitSet, input, indicator,
+          groupSet, groupSets, aggCalls);
     } catch (InvalidRelException e) {
       // Semantic error not possible. Must be a bug. Convert to
       // internal error.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6d79b5eb/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoRules.java
----------------------------------------------------------------------
diff --git a/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoRules.java b/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoRules.java
index 88a45db..2b2f250 100644
--- a/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoRules.java
+++ b/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoRules.java
@@ -455,7 +455,9 @@ public class MongoRules {
             rel.getCluster(),
             traitSet,
             convert(agg.getInput(), traitSet),
+            agg.indicator,
             agg.getGroupSet(),
+            agg.getGroupSets(),
             agg.getAggCallList());
       } catch (InvalidRelException e) {
         LOGGER.warning(e.toString());


[19/58] [abbrv] [partial] incubator-calcite git commit: [CALCITE-306] Standardize code style for "import package.*; "

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/javac/JaninoCompiler.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/javac/JaninoCompiler.java b/core/src/main/java/org/apache/calcite/util/javac/JaninoCompiler.java
index 8ce10a9..e508d7d 100644
--- a/core/src/main/java/org/apache/calcite/util/javac/JaninoCompiler.java
+++ b/core/src/main/java/org/apache/calcite/util/javac/JaninoCompiler.java
@@ -14,18 +14,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.javac;
+package org.apache.calcite.util.javac;
+
+import org.apache.calcite.util.Util;
+
+import org.codehaus.janino.JavaSourceClassLoader;
+import org.codehaus.janino.util.ClassFile;
+import org.codehaus.janino.util.resource.MapResourceFinder;
+import org.codehaus.janino.util.resource.ResourceFinder;
 
 import java.io.File;
 import java.io.FileOutputStream;
 import java.io.IOException;
-import java.util.*;
-
-import org.eigenbase.util.*;
-
-import org.codehaus.janino.*;
-import org.codehaus.janino.util.*;
-import org.codehaus.janino.util.resource.*;
+import java.util.HashMap;
+import java.util.Map;
 
 /**
  * <code>JaninoCompiler</code> implements the {@link JavaCompiler} interface by
@@ -34,7 +36,7 @@ import org.codehaus.janino.util.resource.*;
 public class JaninoCompiler implements JavaCompiler {
   //~ Instance fields --------------------------------------------------------
 
-  private JaninoCompilerArgs args = new JaninoCompilerArgs();
+  public JaninoCompilerArgs args = new JaninoCompilerArgs();
 
   // REVIEW jvs 28-June-2004:  pool this instance?  Is it thread-safe?
   private AccountingClassLoader classLoader;
@@ -103,7 +105,7 @@ public class JaninoCompiler implements JavaCompiler {
   /**
    * Arguments to an invocation of the Janino compiler.
    */
-  private static class JaninoCompilerArgs extends JavaCompilerArgs {
+  public static class JaninoCompilerArgs extends JavaCompilerArgs {
     String destdir;
     String fullClassName;
     String source;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/javac/JavaCompiler.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/javac/JavaCompiler.java b/core/src/main/java/org/apache/calcite/util/javac/JavaCompiler.java
index f810292..93cc7dd 100644
--- a/core/src/main/java/org/apache/calcite/util/javac/JavaCompiler.java
+++ b/core/src/main/java/org/apache/calcite/util/javac/JavaCompiler.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.javac;
+package org.apache.calcite.util.javac;
 
 /**
  * The interface <code>JavaCompiler</code> represents an interface to invoke a

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/javac/JavaCompilerArgs.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/javac/JavaCompilerArgs.java b/core/src/main/java/org/apache/calcite/util/javac/JavaCompilerArgs.java
index 1e51f4f..e687a25 100644
--- a/core/src/main/java/org/apache/calcite/util/javac/JavaCompilerArgs.java
+++ b/core/src/main/java/org/apache/calcite/util/javac/JavaCompilerArgs.java
@@ -14,13 +14,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.javac;
+package org.apache.calcite.util.javac;
 
-import java.util.*;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.StringTokenizer;
 
 /**
- * A <code>JavaCompilerArgs</code> holds the arguments for a {@link
- * JavaCompiler}.
+ * A <code>JavaCompilerArgs</code> holds the arguments for a
+ * {@link JavaCompiler}.
  *
  * <p>Specific implementations of {@link JavaCompiler} may override <code>
  * set<i>Argument</i></code> methods to store arguments in a different fashion,
@@ -128,8 +130,8 @@ public class JavaCompilerArgs {
    * <p>This method is optional. It only works if the compiler supports
    * in-memory compilation. If this compiler does not return in-memory
    * compilation (which the base class does not), {@link #supportsSetSource}
-   * returns false, and this method throws {@link
-   * UnsupportedOperationException}.
+   * returns false, and this method throws
+   * {@link UnsupportedOperationException}.
    */
   public void setSource(String source, String fileName) {
     throw new UnsupportedOperationException();

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/javac/package-info.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/javac/package-info.java b/core/src/main/java/org/apache/calcite/util/javac/package-info.java
index 3998ebc..aabcd55 100644
--- a/core/src/main/java/org/apache/calcite/util/javac/package-info.java
+++ b/core/src/main/java/org/apache/calcite/util/javac/package-info.java
@@ -18,6 +18,6 @@
 /**
  * Provides compilers for Java code.
  */
-package org.eigenbase.javac;
+package org.apache.calcite.util.javac;
 
 // End package-info.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/mapping/AbstractSourceMapping.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/mapping/AbstractSourceMapping.java b/core/src/main/java/org/apache/calcite/util/mapping/AbstractSourceMapping.java
index 2737362..167aec7 100644
--- a/core/src/main/java/org/apache/calcite/util/mapping/AbstractSourceMapping.java
+++ b/core/src/main/java/org/apache/calcite/util/mapping/AbstractSourceMapping.java
@@ -14,14 +14,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.util.mapping;
+package org.apache.calcite.util.mapping;
 
 import java.util.Iterator;
 
 /**
  * Simple implementation of
- * {@link org.eigenbase.util.mapping.Mappings.TargetMapping} where the number
- * of sources and targets are specified as constructor parameters and you
+ * {@link org.apache.calcite.util.mapping.Mappings.TargetMapping} where the
+ * number of sources and targets are specified as constructor parameters and you
  * just need to implement one method,
  */
 public abstract class AbstractSourceMapping
@@ -96,4 +96,4 @@ public abstract class AbstractSourceMapping
   public abstract int getSourceOpt(int source);
 }
 
-// End AbstractTargetMapping.java
+// End AbstractSourceMapping.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/mapping/AbstractTargetMapping.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/mapping/AbstractTargetMapping.java b/core/src/main/java/org/apache/calcite/util/mapping/AbstractTargetMapping.java
index 3048601..a1ee5cf 100644
--- a/core/src/main/java/org/apache/calcite/util/mapping/AbstractTargetMapping.java
+++ b/core/src/main/java/org/apache/calcite/util/mapping/AbstractTargetMapping.java
@@ -14,14 +14,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.util.mapping;
+package org.apache.calcite.util.mapping;
 
 import java.util.Iterator;
 
 /**
  * Simple implementation of
- * {@link org.eigenbase.util.mapping.Mappings.TargetMapping} where the number
- * of sources and targets are specified as constructor parameters and you
+ * {@link org.apache.calcite.util.mapping.Mappings.TargetMapping} where the
+ * number of sources and targets are specified as constructor parameters and you
  * just need to implement one method,
  */
 public abstract class AbstractTargetMapping

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/mapping/IntPair.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/mapping/IntPair.java b/core/src/main/java/org/apache/calcite/util/mapping/IntPair.java
index e6e122b..7160f39 100644
--- a/core/src/main/java/org/apache/calcite/util/mapping/IntPair.java
+++ b/core/src/main/java/org/apache/calcite/util/mapping/IntPair.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.util.mapping;
+package org.apache.calcite.util.mapping;
 
 import java.util.AbstractList;
 import java.util.List;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/mapping/Mapping.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/mapping/Mapping.java b/core/src/main/java/org/apache/calcite/util/mapping/Mapping.java
index 5023e08..1a62f42 100644
--- a/core/src/main/java/org/apache/calcite/util/mapping/Mapping.java
+++ b/core/src/main/java/org/apache/calcite/util/mapping/Mapping.java
@@ -14,9 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.util.mapping;
+package org.apache.calcite.util.mapping;
 
-import java.util.*;
+import java.util.Iterator;
 
 /**
  * A <dfn>Mapping</dfn> is a relationship between a source domain to target
@@ -45,8 +45,8 @@ public interface Mapping
   /**
    * Returns an iterator over the elements in this mapping.
    *
-   * <p>This method is optional; implementations may throw {@link
-   * UnsupportedOperationException}.
+   * <p>This method is optional; implementations may throw
+   * {@link UnsupportedOperationException}.
    */
   Iterator<IntPair> iterator();
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/mapping/MappingType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/mapping/MappingType.java b/core/src/main/java/org/apache/calcite/util/mapping/MappingType.java
index 5498055..c7cee41 100644
--- a/core/src/main/java/org/apache/calcite/util/mapping/MappingType.java
+++ b/core/src/main/java/org/apache/calcite/util/mapping/MappingType.java
@@ -14,14 +14,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.util.mapping;
+package org.apache.calcite.util.mapping;
 
 /**
- * Describes the type of a mapping, from the most general {@link #MULTI_FUNCTION}
- * (every element in the source and target domain can participate in many
- * mappings) to the most retricted {@link #BIJECTION} (every element in the
- * source and target domain must be paired with precisely one element in the
- * other domain).
+ * Describes the type of a mapping, from the most general
+ * {@link #MULTI_FUNCTION} (every element in the source and target domain can
+ * participate in many mappings) to the most retricted {@link #BIJECTION} (every
+ * element in the source and target domain must be paired with precisely one
+ * element in the other domain).
  *
  * <p>Some common types:
  *
@@ -36,9 +36,9 @@ package org.eigenbase.util.mapping;
  * Every source has precisely one target, and vice versa.
  * </ul>
  *
- * <p>Once you know what type of mapping you want, call {@link
- * Mappings#create(MappingType, int, int)} to create an efficient implementation
- * of that mapping.
+ * <p>Once you know what type of mapping you want, call
+ * {@link Mappings#create(MappingType, int, int)} to create an efficient
+ * implementation of that mapping.
  */
 public enum MappingType {
   //            ordinal source target function inverse
@@ -92,8 +92,8 @@ public enum MappingType {
    * An inverse function has a source for every target, but a source might
    * have 0, 1 or more targets.
    *
-   * <p>Obeys the constaints {@link MappingType#isMandatorySource()}, {@link
-   * MappingType#isSingleSource()}.
+   * <p>Obeys the constaints {@link MappingType#isMandatorySource()},
+   * {@link MappingType#isSingleSource()}.
    *
    * <p>Similar types:
    *

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/mapping/Mappings.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/mapping/Mappings.java b/core/src/main/java/org/apache/calcite/util/mapping/Mappings.java
index 071fdb4..8d9733b 100644
--- a/core/src/main/java/org/apache/calcite/util/mapping/Mappings.java
+++ b/core/src/main/java/org/apache/calcite/util/mapping/Mappings.java
@@ -14,22 +14,29 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.util.mapping;
+package org.apache.calcite.util.mapping;
 
-import java.util.*;
-
-import org.eigenbase.util.*;
-
-import net.hydromatic.optiq.util.BitSets;
+import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.IntList;
+import org.apache.calcite.util.Permutation;
+import org.apache.calcite.util.Util;
 
 import com.google.common.base.Function;
 
+import java.util.AbstractList;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.BitSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
 /**
  * Utility functions related to mappings.
  *
  * @see MappingType
  * @see Mapping
- * @see Permutation
+ * @see org.apache.calcite.util.Permutation
  */
 public abstract class Mappings {
   //~ Constructors -----------------------------------------------------------
@@ -191,8 +198,8 @@ public abstract class Mappings {
   public static <T> List<T> apply(final Mapping mapping, final List<T> list) {
     if (mapping.getSourceCount() != list.size()) {
       // REVIEW: too strict?
-      throw new IllegalArgumentException(
-          "mapping source count " + mapping.getSourceCount()
+      throw new IllegalArgumentException("mapping source count "
+          + mapping.getSourceCount()
           + " does not match list size " + list.size());
     }
     final int targetCount = mapping.getTargetCount();
@@ -231,13 +238,11 @@ public abstract class Mappings {
       final Mapping mapping,
       final List<T> list) {
     return new AbstractList<T>() {
-      @Override
-      public T get(int index) {
+      @Override public T get(int index) {
         return list.get(mapping.getSource(index));
       }
 
-      @Override
-      public int size() {
+      @Override public int size() {
         return mapping.getTargetCount();
       }
     };
@@ -660,7 +665,7 @@ public abstract class Mappings {
   }
 
   /** Inverts an {@link java.lang.Iterable} over
-   * {@link org.eigenbase.util.mapping.IntPair}s. */
+   * {@link org.apache.calcite.util.mapping.IntPair}s. */
   public static Iterable<IntPair> invert(final Iterable<IntPair> pairs) {
     return new Iterable<IntPair>() {
       public Iterator<IntPair> iterator() {
@@ -670,7 +675,7 @@ public abstract class Mappings {
   }
 
   /** Inverts an {@link java.util.Iterator} over
-   * {@link org.eigenbase.util.mapping.IntPair}s. */
+   * {@link org.apache.calcite.util.mapping.IntPair}s. */
   public static Iterator<IntPair> invert(final Iterator<IntPair> pairs) {
     return new Iterator<IntPair>() {
       public boolean hasNext() {
@@ -797,6 +802,7 @@ public abstract class Mappings {
 
   //~ Inner Classes ----------------------------------------------------------
 
+  /** Abstract implementation of {@link Mapping}. */
   public abstract static class AbstractMapping implements Mapping {
     public void set(int source, int target) {
       throw new UnsupportedOperationException();
@@ -911,6 +917,8 @@ public abstract class Mappings {
     }
   }
 
+  /** Abstract implementation of mapping where both source and target
+   * domains are finite. */
   public abstract static class FiniteAbstractMapping extends AbstractMapping {
     public Iterator<IntPair> iterator() {
       return new FunctionMappingIter(this);
@@ -928,6 +936,8 @@ public abstract class Mappings {
     }
   }
 
+  /** Iterator that yields the (source, target) values in a
+   * {@link FunctionMapping}. */
   static class FunctionMappingIter implements Iterator<IntPair> {
     private int i = 0;
     private final FunctionMapping mapping;
@@ -1070,8 +1080,8 @@ public abstract class Mappings {
      * @param sourceList  List whose i'th element is the source of target #i
      * @param sourceCount Number of elements in the source domain
      * @param mappingType Mapping type, must be
-     *                    {@link org.eigenbase.util.mapping.MappingType#PARTIAL_SURJECTION}
-     *                    or stronger.
+     *   {@link org.apache.calcite.util.mapping.MappingType#PARTIAL_SURJECTION}
+     *   or stronger.
      */
     public PartialMapping(
         List<Integer> sourceList,
@@ -1194,6 +1204,7 @@ public abstract class Mappings {
       return true;
     }
 
+    /** Mapping iterator. */
     private class MappingItr implements Iterator<IntPair> {
       int i = -1;
 
@@ -1252,8 +1263,7 @@ public abstract class Mappings {
       assert isValid();
       final int prevTarget = targets[source];
       if (prevTarget != -1) {
-        throw new IllegalArgumentException(
-            "source #" + source
+        throw new IllegalArgumentException("source #" + source
             + " is already mapped to target #" + target);
       }
       targets[source] = target;
@@ -1265,6 +1275,7 @@ public abstract class Mappings {
     }
   }
 
+  /** The identity mapping, of a given size, or infinite. */
   public static class IdentityMapping extends AbstractMapping
       implements FunctionMapping, TargetMapping, SourceMapping {
     private final int size;
@@ -1346,6 +1357,8 @@ public abstract class Mappings {
     }
   }
 
+  /** Source mapping that returns the same result as a parent
+   * {@link SourceMapping} except for specific overriding elements. */
   public static class OverridingSourceMapping extends AbstractMapping
       implements SourceMapping {
     private final SourceMapping parent;
@@ -1403,6 +1416,8 @@ public abstract class Mappings {
     }
   }
 
+  /** Target mapping that returns the same result as a parent
+   * {@link TargetMapping} except for specific overriding elements. */
   public static class OverridingTargetMapping extends AbstractMapping
       implements TargetMapping {
     private final TargetMapping parent;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/mapping/package-info.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/mapping/package-info.java b/core/src/main/java/org/apache/calcite/util/mapping/package-info.java
index ac108b5..2a994ca 100644
--- a/core/src/main/java/org/apache/calcite/util/mapping/package-info.java
+++ b/core/src/main/java/org/apache/calcite/util/mapping/package-info.java
@@ -18,6 +18,6 @@
 /**
  * Support for algebraic maps.
  */
-package org.eigenbase.util.mapping;
+package org.apache.calcite.util.mapping;
 
 // End package-info.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/package-info.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/package-info.java b/core/src/main/java/org/apache/calcite/util/package-info.java
index 290d281..ba2185f 100644
--- a/core/src/main/java/org/apache/calcite/util/package-info.java
+++ b/core/src/main/java/org/apache/calcite/util/package-info.java
@@ -18,6 +18,6 @@
 /**
  * Provides utility classes.
  */
-package net.hydromatic.optiq.util;
+package org.apache.calcite.util;
 
 // End package-info.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/trace/CalciteLogger.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/trace/CalciteLogger.java b/core/src/main/java/org/apache/calcite/util/trace/CalciteLogger.java
index 67cd4fd..8105f8e 100644
--- a/core/src/main/java/org/apache/calcite/util/trace/CalciteLogger.java
+++ b/core/src/main/java/org/apache/calcite/util/trace/CalciteLogger.java
@@ -14,32 +14,44 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.trace;
+package org.apache.calcite.util.trace;
 
-import java.util.logging.*;
+import java.util.logging.Level;
+import java.util.logging.Logger;
 
 /**
- * This class is a small extension to {@link Logger}. {@link Logger#log(Level
- * level, String msg, Object[] params)} is expensive to call, since the caller
- * must always allocate and fill in the array <code>params</code>, even when
- * <code>level</code> will prevent a message being logged. On the other hand,
- * {@link Logger#log(Level level, String msg)} and {@link Logger#log(Level
- * level, String msg, Object)} do not have this problem. As a workaround this
- * class provides {@link #log(Level, String msg, Object, Object)} etc. (The
- * varargs feature of java 1.5 half-solves this problem, by automatically
- * wrapping args in an array, but it does so without testing the level.) Usage:
- * replace: <code>static final Logger tracer =
- * EigenbaseTracer.getMyTracer();</code> by: <code>static final EigenbaseLogger
- * tracer = new EigenbaseLogger(EigenbaseTracer.getMyTracer());</code>
+ * Small extension to {@link Logger} with some performance improvements.
+ *
+ * <p>{@link Logger#log(Level level, String msg, Object[] params)} is expensive
+ * to call, since the caller must always allocate and fill in the array
+ * <code>params</code>, even when <code>level</code> will prevent a message
+ * being logged. On the other hand, {@link Logger#log(Level level, String msg)}
+ * and {@link Logger#log(Level level, String msg, Object)} do not have this
+ * problem.
+ *
+ * <p>As a workaround this class provides
+ * {@link #log(Level, String msg, Object, Object)} etc. (The varargs feature of
+ * java 1.5 half-solves this problem, by automatically wrapping args in an
+ * array, but it does so without testing the level.)
+ *
+ * <p>Usage: replace:
+ *
+ * <blockquote><code>static final Logger tracer =
+ * CalciteTracer.getMyTracer();</code></blockquote>
+ *
+ * <p>by:
+ *
+ * <blockquote><code>static final CalciteLogger tracer =
+ *     new CalciteLogger(CalciteTrace.getMyTracer());</code></blockquote>
  */
-public class EigenbaseLogger {
+public class CalciteLogger {
   //~ Instance fields --------------------------------------------------------
 
   private final Logger logger; // delegate
 
   //~ Constructors -----------------------------------------------------------
 
-  public EigenbaseLogger(Logger logger) {
+  public CalciteLogger(Logger logger) {
     assert logger != null;
     this.logger = logger;
   }
@@ -144,4 +156,4 @@ public class EigenbaseLogger {
   }
 }
 
-// End EigenbaseLogger.java
+// End CalciteLogger.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/trace/CalciteTimingTracer.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/trace/CalciteTimingTracer.java b/core/src/main/java/org/apache/calcite/util/trace/CalciteTimingTracer.java
index c89b233..bd962ba 100644
--- a/core/src/main/java/org/apache/calcite/util/trace/CalciteTimingTracer.java
+++ b/core/src/main/java/org/apache/calcite/util/trace/CalciteTimingTracer.java
@@ -14,17 +14,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.trace;
+package org.apache.calcite.util.trace;
 
-import java.text.*;
-
-import java.util.logging.*;
+import java.text.DecimalFormat;
+import java.util.logging.Level;
+import java.util.logging.Logger;
 
 /**
- * EigenbaseTimingTracer provides a mechanism for tracing the timing of a call
+ * CalciteTimingTracer provides a mechanism for tracing the timing of a call
  * sequence at nanosecond resolution.
  */
-public class EigenbaseTimingTracer {
+public class CalciteTimingTracer {
   //~ Static fields/initializers ---------------------------------------------
 
   private static final DecimalFormat DECIMAL_FORMAT =
@@ -46,7 +46,7 @@ public class EigenbaseTimingTracer {
    *                   used
    * @param startEvent event to trace as start of timing
    */
-  public EigenbaseTimingTracer(
+  public CalciteTimingTracer(
       Logger logger,
       String startEvent) {
     if (!logger.isLoggable(Level.FINE)) {
@@ -78,4 +78,4 @@ public class EigenbaseTimingTracer {
   }
 }
 
-// End EigenbaseTimingTracer.java
+// End CalciteTimingTracer.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/trace/CalciteTrace.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/trace/CalciteTrace.java b/core/src/main/java/org/apache/calcite/util/trace/CalciteTrace.java
index 1ab124a..2f69f21 100644
--- a/core/src/main/java/org/apache/calcite/util/trace/CalciteTrace.java
+++ b/core/src/main/java/org/apache/calcite/util/trace/CalciteTrace.java
@@ -14,28 +14,26 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.trace;
+package org.apache.calcite.util.trace;
 
-import java.io.File;
-import java.util.logging.*;
-
-import org.eigenbase.rel.RelImplementorImpl;
-import org.eigenbase.relopt.*;
-import org.eigenbase.util.property.*;
-
-import net.hydromatic.linq4j.function.Function2;
-import net.hydromatic.linq4j.function.Functions;
+import org.apache.calcite.linq4j.function.Function2;
+import org.apache.calcite.linq4j.function.Functions;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.prepare.Prepare;
+import org.apache.calcite.rel.RelImplementorImpl;
 
-import net.hydromatic.optiq.prepare.Prepare;
+import java.io.File;
+import java.util.logging.Level;
+import java.util.logging.Logger;
 
 /**
  * Contains all of the {@link java.util.logging.Logger tracers} used within
- * org.eigenbase class libraries.
+ * org.apache.calcite class libraries.
  *
  * <h3>Note to developers</h3>
  *
- * <p>Please ensure that every tracer used in org.eigenbase is added to this
- * class as a <em>public static final</em> member called <code>
+ * <p>Please ensure that every tracer used in org.apache.calcite is added to
+ * this class as a <em>public static final</em> member called <code>
  * <i>component</i>Tracer</code>. For example, {@link #getPlannerTracer} is the
  * tracer used by all classes which take part in the query planning process.
  *
@@ -48,21 +46,20 @@ import net.hydromatic.optiq.prepare.Prepare;
  * perhaps <em>protected</em>) <em>static final</em> member called <code>
  * tracer</code>.
  */
-public abstract class EigenbaseTrace {
+public abstract class CalciteTrace {
   //~ Static fields/initializers ---------------------------------------------
 
   /**
-   * The "org.eigenbase.sql.parser" tracer reports parser events in {@link
-   * org.eigenbase.sql.parser.SqlParser} and other classes (at level {@link
-   * Level#FINE} or higher).
+   * The "org.apache.calcite.sql.parser" tracer reports parser events in
+   * {@link org.apache.calcite.sql.parser.SqlParser} and other classes (at level
+   * {@link Level#FINE} or higher).
    */
   public static final Logger PARSER_LOGGER = getParserTracer();
 
   private static final ThreadLocal<Function2<Void, File, String>>
   DYNAMIC_HANDLER =
       new ThreadLocal<Function2<Void, File, String>>() {
-        @Override
-        protected Function2<Void, File, String> initialValue() {
+        @Override protected Function2<Void, File, String> initialValue() {
           return Functions.ignore2();
         }
       };
@@ -70,7 +67,7 @@ public abstract class EigenbaseTrace {
   //~ Methods ----------------------------------------------------------------
 
   /**
-   * The "org.eigenbase.relopt.RelOptPlanner" tracer prints the query
+   * The "org.apache.calcite.plan.RelOptPlanner" tracer prints the query
    * optimization process.
    *
    * <p>Levels:
@@ -87,7 +84,7 @@ public abstract class EigenbaseTrace {
   }
 
   /**
-   * The "net.hydromatic.optiq.prepare.Prepare" tracer prints the generated
+   * The "org.apache.calcite.prepare.Prepare" tracer prints the generated
    * program at level {@link java.util.logging.Level#FINE} or higher.
    */
   public static Logger getStatementTracer() {
@@ -95,7 +92,7 @@ public abstract class EigenbaseTrace {
   }
 
   /**
-   * The "org.eigenbase.rel.RelImplementorImpl" tracer reports when
+   * The "org.apache.calcite.rel.RelImplementorImpl" tracer reports when
    * expressions are bound to variables ({@link Level#FINE})
    */
   public static Logger getRelImplementorTracer() {
@@ -103,42 +100,27 @@ public abstract class EigenbaseTrace {
   }
 
   /**
-   * The tracer "org.eigenbase.sql.timing" traces timing for various stages of
-   * query processing.
+   * The tracer "org.apache.calcite.sql.timing" traces timing for
+   * various stages of query processing.
    *
-   * @see EigenbaseTimingTracer
+   * @see CalciteTimingTracer
    */
   public static Logger getSqlTimingTracer() {
-    return Logger.getLogger("org.eigenbase.sql.timing");
+    return Logger.getLogger("org.apache.calcite.sql.timing");
   }
 
   /**
-   * The "org.eigenbase.sql.parser" tracer reports parse events.
+   * The "org.apache.calcite.sql.parser" tracer reports parse events.
    */
   public static Logger getParserTracer() {
-    return Logger.getLogger("org.eigenbase.sql.parser");
+    return Logger.getLogger("org.apache.calcite.sql.parser");
   }
 
   /**
-   * The "org.eigenbase.sql2rel" tracer reports parse events.
+   * The "org.apache.calcite.sql2rel" tracer reports parse events.
    */
   public static Logger getSqlToRelTracer() {
-    return Logger.getLogger("org.eigenbase.sql2rel");
-  }
-
-  /**
-   * The "org.eigenbase.jmi.JmiChangeSet" tracer reports JmiChangeSet events.
-   */
-  public static Logger getJmiChangeSetTracer() {
-    return Logger.getLogger("org.eigenbase.jmi.JmiChangeSet");
-  }
-
-  /**
-   * The "org.eigenbase.util.property.Property" tracer reports errors related
-   * to all manner of properties.
-   */
-  public static Logger getPropertyTracer() {
-    return Logger.getLogger(Property.class.getName());
+    return Logger.getLogger("org.apache.calcite.sql2rel");
   }
 
   /**
@@ -151,4 +133,4 @@ public abstract class EigenbaseTrace {
   }
 }
 
-// End EigenbaseTrace.java
+// End CalciteTrace.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/trace/package-info.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/trace/package-info.java b/core/src/main/java/org/apache/calcite/util/trace/package-info.java
index 7517297..282bbc9 100644
--- a/core/src/main/java/org/apache/calcite/util/trace/package-info.java
+++ b/core/src/main/java/org/apache/calcite/util/trace/package-info.java
@@ -18,6 +18,6 @@
 /**
  * Tracing services.
  */
-package org.eigenbase.trace;
+package org.apache.calcite.util.trace;
 
 // End package-info.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/resources/META-INF/services/java.sql.Driver
----------------------------------------------------------------------
diff --git a/core/src/main/resources/META-INF/services/java.sql.Driver b/core/src/main/resources/META-INF/services/java.sql.Driver
index 1113e3e..c99c3fc 100644
--- a/core/src/main/resources/META-INF/services/java.sql.Driver
+++ b/core/src/main/resources/META-INF/services/java.sql.Driver
@@ -1 +1 @@
-net.hydromatic.optiq.jdbc.Driver
\ No newline at end of file
+org.apache.calcite.jdbc.Driver

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
----------------------------------------------------------------------
diff --git a/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties b/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
index 6f6eb5d..b2c6b0a 100644
--- a/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
+++ b/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
@@ -1,5 +1,5 @@
-# Resources for the eigenbase project.
-# See wrapper class org.eigenbase.resource.EigenbaseResource.
+# Resources for the Apache Calcite project.
+# See wrapper class org.apache.calcite.runtime.CalciteResource.
 #
 # Licensed to the Apache Software Foundation (ASF) under one or more
 # contributor license agreements.  See the NOTICE file distributed with
@@ -176,4 +176,4 @@ RequireDefaultConstructor=Declaring class ''{0}'' of non-static user-defined fun
 FirstParameterOfAdd=In user-defined aggregate class ''{0}'', first parameter to ''add'' method must be the accumulator (the return type of the ''init'' method)
 FilterableTableInventedFilter=FilterableTable.scan returned a filter that was not in the original list: {0}
 FilterableTableScanReturnedNull=FilterableTable.scan must not return null
-# End org.eigenbase.resource.EigenbaseResource.properties
+# End CalciteResource.properties

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/adapter/clone/ArrayTableTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/adapter/clone/ArrayTableTest.java b/core/src/test/java/org/apache/calcite/adapter/clone/ArrayTableTest.java
index 534e9c2..6136e54 100644
--- a/core/src/test/java/org/apache/calcite/adapter/clone/ArrayTableTest.java
+++ b/core/src/test/java/org/apache/calcite/adapter/clone/ArrayTableTest.java
@@ -14,22 +14,23 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl.clone;
+package org.apache.calcite.adapter.clone;
 
-import net.hydromatic.linq4j.Enumerable;
-import net.hydromatic.linq4j.Linq4j;
-
-import net.hydromatic.optiq.jdbc.JavaTypeFactoryImpl;
-
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.reltype.RelDataTypeImpl;
-import org.eigenbase.reltype.RelDataTypeSystem;
+import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
+import org.apache.calcite.linq4j.Enumerable;
+import org.apache.calcite.linq4j.Linq4j;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeImpl;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
 
 import org.junit.Test;
 
 import java.util.Arrays;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
 
 /**
  * Unit test for {@link ArrayTable} and {@link ColumnLoader}.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/adapter/generate/RangeTable.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/adapter/generate/RangeTable.java b/core/src/test/java/org/apache/calcite/adapter/generate/RangeTable.java
index 4bba4e9..16109a1 100644
--- a/core/src/test/java/org/apache/calcite/adapter/generate/RangeTable.java
+++ b/core/src/test/java/org/apache/calcite/adapter/generate/RangeTable.java
@@ -14,20 +14,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl.generate;
-
-import net.hydromatic.linq4j.Enumerator;
-import net.hydromatic.linq4j.QueryProvider;
-import net.hydromatic.linq4j.Queryable;
-
-import net.hydromatic.optiq.SchemaPlus;
-import net.hydromatic.optiq.TableFactory;
-import net.hydromatic.optiq.impl.AbstractTableQueryable;
-import net.hydromatic.optiq.impl.java.AbstractQueryableTable;
-
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.reltype.RelDataTypeFactory;
-import org.eigenbase.sql.type.SqlTypeName;
+package org.apache.calcite.adapter.generate;
+
+import org.apache.calcite.adapter.java.AbstractQueryableTable;
+import org.apache.calcite.linq4j.Enumerator;
+import org.apache.calcite.linq4j.QueryProvider;
+import org.apache.calcite.linq4j.Queryable;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.schema.TableFactory;
+import org.apache.calcite.schema.impl.AbstractTableQueryable;
+import org.apache.calcite.sql.type.SqlTypeName;
 
 import java.util.Map;
 import java.util.NoSuchElementException;
@@ -94,9 +92,9 @@ public class RangeTable extends AbstractQueryableTable {
     };
   }
 
-  /** Implementation of {@link net.hydromatic.optiq.TableFactory} that allows
-   * a {@link RangeTable} to be included as a custom table in a Calcite model
-   * file. */
+  /** Implementation of {@link org.apache.calcite.schema.TableFactory} that
+   * allows a {@link RangeTable} to be included as a custom table in a Calcite
+   * model file. */
   public static class Factory implements TableFactory<RangeTable> {
     public RangeTable create(
         SchemaPlus schema,

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/examples/foodmart/java/JdbcExample.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/examples/foodmart/java/JdbcExample.java b/core/src/test/java/org/apache/calcite/examples/foodmart/java/JdbcExample.java
index 65dbcd5..cf31244 100644
--- a/core/src/test/java/org/apache/calcite/examples/foodmart/java/JdbcExample.java
+++ b/core/src/test/java/org/apache/calcite/examples/foodmart/java/JdbcExample.java
@@ -14,13 +14,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.examples.foodmart.java;
+package org.apache.calcite.examples.foodmart.java;
 
-import net.hydromatic.optiq.SchemaPlus;
-import net.hydromatic.optiq.impl.java.ReflectiveSchema;
-import net.hydromatic.optiq.jdbc.OptiqConnection;
+import org.apache.calcite.adapter.java.ReflectiveSchema;
+import org.apache.calcite.jdbc.CalciteConnection;
+import org.apache.calcite.schema.SchemaPlus;
 
-import java.sql.*;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
 
 /**
  * Example of using Calcite via JDBC.
@@ -33,18 +37,17 @@ public class JdbcExample {
   }
 
   public void run() throws ClassNotFoundException, SQLException {
-    Class.forName("net.hydromatic.optiq.jdbc.Driver");
+    Class.forName("org.apache.calcite.jdbc.Driver");
     Connection connection =
         DriverManager.getConnection("jdbc:calcite:");
-    OptiqConnection optiqConnection =
-        connection.unwrap(OptiqConnection.class);
-    SchemaPlus rootSchema = optiqConnection.getRootSchema();
+    CalciteConnection calciteConnection =
+        connection.unwrap(CalciteConnection.class);
+    SchemaPlus rootSchema = calciteConnection.getRootSchema();
     rootSchema.add("hr", new ReflectiveSchema(new Hr()));
     rootSchema.add("foodmart", new ReflectiveSchema(new Foodmart()));
     Statement statement = connection.createStatement();
     ResultSet resultSet =
-        statement.executeQuery(
-            "select *\n"
+        statement.executeQuery("select *\n"
             + "from \"foodmart\".\"sales_fact_1997\" as s\n"
             + "join \"hr\".\"emps\" as e\n"
             + "on e.\"empid\" = s.\"cust_id\"");
@@ -65,6 +68,7 @@ public class JdbcExample {
     connection.close();
   }
 
+  /** Object that will be used via reflection to create the "hr" schema. */
   public static class Hr {
     public final Employee[] emps = {
       new Employee(100, "Bill"),
@@ -73,6 +77,7 @@ public class JdbcExample {
     };
   }
 
+  /** Object that will be used via reflection to create the "emps" table. */
   public static class Employee {
     public final int empid;
     public final String name;
@@ -83,6 +88,8 @@ public class JdbcExample {
     }
   }
 
+  /** Object that will be used via reflection to create the "foodmart"
+   * schema. */
   public static class Foodmart {
     public final SalesFact[] sales_fact_1997 = {
       new SalesFact(100, 10),
@@ -90,6 +97,9 @@ public class JdbcExample {
     };
   }
 
+
+  /** Object that will be used via reflection to create the
+   * "sales_fact_1997" fact table. */
   public static class SalesFact {
     public final int cust_id;
     public final int prod_id;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/plan/RelOptPlanReaderTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/plan/RelOptPlanReaderTest.java b/core/src/test/java/org/apache/calcite/plan/RelOptPlanReaderTest.java
index e69b82b..8c95301 100644
--- a/core/src/test/java/org/apache/calcite/plan/RelOptPlanReaderTest.java
+++ b/core/src/test/java/org/apache/calcite/plan/RelOptPlanReaderTest.java
@@ -14,35 +14,39 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.relopt;
+package org.apache.calcite.plan;
 
-import org.eigenbase.rel.*;
-
-import net.hydromatic.optiq.impl.jdbc.JdbcRules;
+import org.apache.calcite.adapter.jdbc.JdbcRules;
+import org.apache.calcite.rel.AbstractRelNode;
+import org.apache.calcite.rel.externalize.RelJson;
+import org.apache.calcite.rel.logical.LogicalProject;
 
 import org.junit.Test;
 
-import static org.hamcrest.CoreMatchers.*;
-import static org.junit.Assert.*;
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.CoreMatchers.sameInstance;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
 
 /**
- * Unit test for {@link org.eigenbase.rel.RelJson}.
+ * Unit test for {@link org.apache.calcite.rel.externalize.RelJson}.
  */
 public class RelOptPlanReaderTest {
   @Test public void testTypeToClass() {
     RelJson relJson = new RelJson(null);
 
-    // in org.eigenbase.rel package
-    assertThat(relJson.classToTypeName(ProjectRel.class),
-        is("ProjectRel"));
-    assertThat(relJson.typeNameToClass("ProjectRel"),
-        sameInstance((Class) ProjectRel.class));
+    // in org.apache.calcite.rel package
+    assertThat(relJson.classToTypeName(LogicalProject.class),
+        is("LogicalProject"));
+    assertThat(relJson.typeNameToClass("LogicalProject"),
+        sameInstance((Class) LogicalProject.class));
 
-    // in net.hydromatic.optiq.impl.jdbc.JdbcRules outer class
-    assertThat(relJson.classToTypeName(JdbcRules.JdbcProjectRel.class),
+    // in org.apache.calcite.adapter.jdbc.JdbcRules outer class
+    assertThat(relJson.classToTypeName(JdbcRules.JdbcProject.class),
         is("JdbcProjectRel"));
     assertThat(relJson.typeNameToClass("JdbcProjectRel"),
-        equalTo((Class) JdbcRules.JdbcProjectRel.class));
+        equalTo((Class) JdbcRules.JdbcProject.class));
 
     try {
       Class clazz = relJson.typeNameToClass("NonExistentRel");
@@ -52,16 +56,16 @@ public class RelOptPlanReaderTest {
     }
     try {
       Class clazz =
-          relJson.typeNameToClass("org.eigenbase.rel.NonExistentRel");
+          relJson.typeNameToClass("org.apache.calcite.rel.NonExistentRel");
       fail("expected exception, got " + clazz);
     } catch (RuntimeException e) {
       assertThat(e.getMessage(),
-          is("unknown type org.eigenbase.rel.NonExistentRel"));
+          is("unknown type org.apache.calcite.rel.NonExistentRel"));
     }
 
     // In this class; no special treatment. Note: '$MyRel' not '.MyRel'.
     assertThat(relJson.classToTypeName(MyRel.class),
-        is("org.eigenbase.relopt.RelOptPlanReaderTest$MyRel"));
+        is("org.apache.calcite.plan.RelOptPlanReaderTest$MyRel"));
     assertThat(relJson.typeNameToClass(MyRel.class.getName()),
         equalTo((Class) MyRel.class));
 
@@ -72,10 +76,12 @@ public class RelOptPlanReaderTest {
       fail("expected exception, got " + clazz);
     } catch (RuntimeException e) {
       assertThat(e.getMessage(),
-          is("unknown type org.eigenbase.relopt.RelOptPlanReaderTest.MyRel"));
+          is(
+              "unknown type org.apache.calcite.plan.RelOptPlanReaderTest.MyRel"));
     }
   }
 
+  /** Dummy relational expression. */
   public static class MyRel extends AbstractRelNode {
     public MyRel(RelOptCluster cluster, RelTraitSet traitSet) {
       super(cluster, traitSet);

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/plan/RelOptUtilTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/plan/RelOptUtilTest.java b/core/src/test/java/org/apache/calcite/plan/RelOptUtilTest.java
index 65e8b5f..4f9b8cd 100644
--- a/core/src/test/java/org/apache/calcite/plan/RelOptUtilTest.java
+++ b/core/src/test/java/org/apache/calcite/plan/RelOptUtilTest.java
@@ -14,15 +14,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.relopt;
+package org.apache.calcite.plan;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.util.*;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.sql.type.SqlTypeFactoryImpl;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.util.TestUtil;
+import org.apache.calcite.util.Util;
 
 import org.junit.Test;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
 
 /**
  * Unit test for {@link RelOptUtil} and other classes in this package.
@@ -77,9 +82,8 @@ public class RelOptUtilTest {
       Util.discard(RelOptRule.guessDescription("com.foo.Bar$1"));
       fail("expected exception");
     } catch (RuntimeException e) {
-      assertEquals(
-          "Derived description of rule class com.foo.Bar$1 is an "
-          + "integer, not valid. Supply a description manually.",
+      assertEquals("Derived description of rule class com.foo.Bar$1 is an "
+              + "integer, not valid. Supply a description manually.",
           e.getMessage());
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/plan/RelWriterTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/plan/RelWriterTest.java b/core/src/test/java/org/apache/calcite/plan/RelWriterTest.java
index 6109ca8..5ef1877 100644
--- a/core/src/test/java/org/apache/calcite/plan/RelWriterTest.java
+++ b/core/src/test/java/org/apache/calcite/plan/RelWriterTest.java
@@ -14,43 +14,47 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.relopt;
+package org.apache.calcite.plan;
 
-import java.io.IOException;
-import java.math.BigDecimal;
-import java.util.Arrays;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.rex.RexBuilder;
-import org.eigenbase.sql.SqlExplainLevel;
-import org.eigenbase.sql.fun.SqlStdOperatorTable;
-import org.eigenbase.sql.type.SqlTypeName;
-import org.eigenbase.util.Util;
-
-import net.hydromatic.optiq.SchemaPlus;
-import net.hydromatic.optiq.impl.java.ReflectiveSchema;
-import net.hydromatic.optiq.test.JdbcTest;
-import net.hydromatic.optiq.tools.Frameworks;
-import net.hydromatic.optiq.util.BitSets;
+import org.apache.calcite.adapter.java.ReflectiveSchema;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.externalize.RelJsonReader;
+import org.apache.calcite.rel.externalize.RelJsonWriter;
+import org.apache.calcite.rel.logical.LogicalAggregate;
+import org.apache.calcite.rel.logical.LogicalFilter;
+import org.apache.calcite.rel.logical.LogicalTableScan;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.test.JdbcTest;
+import org.apache.calcite.tools.Frameworks;
+import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
 
 import org.junit.Test;
 
-import static org.hamcrest.CoreMatchers.*;
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.util.Arrays;
+
+import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertThat;
 
 /**
- * Unit test for {@link org.eigenbase.rel.RelJson}.
+ * Unit test for {@link org.apache.calcite.rel.externalize.RelJson}.
  */
 public class RelWriterTest {
-  public static final String XX =
-      "{\n"
+  public static final String XX = "{\n"
       + "  rels: [\n"
       + "    {\n"
       + "      id: \"0\",\n"
-      + "      relOp: \"TableAccessRel\",\n"
+      + "      relOp: \"LogicalTableScan\",\n"
       + "      table: [\n"
       + "        \"hr\",\n"
       + "        \"emps\"\n"
@@ -59,7 +63,7 @@ public class RelWriterTest {
       + "    },\n"
       + "    {\n"
       + "      id: \"1\",\n"
-      + "      relOp: \"FilterRel\",\n"
+      + "      relOp: \"LogicalFilter\",\n"
       + "      condition: {\n"
       + "        op: \"=\",\n"
       + "        operands: [\n"
@@ -72,7 +76,7 @@ public class RelWriterTest {
       + "    },\n"
       + "    {\n"
       + "      id: \"2\",\n"
-      + "      relOp: \"AggregateRel\",\n"
+      + "      relOp: \"LogicalAggregate\",\n"
       + "      group: [\n"
       + "        0\n"
       + "      ],\n"
@@ -103,7 +107,7 @@ public class RelWriterTest {
       + "}";
 
   /**
-   * Unit test for {@link org.eigenbase.rel.RelJsonWriter} on
+   * Unit test for {@link org.apache.calcite.rel.externalize.RelJsonWriter} on
    * a simple tree of relational expressions, consisting of a table, a filter
    * and an aggregate node.
    */
@@ -115,13 +119,13 @@ public class RelWriterTest {
                   RelOptSchema relOptSchema, SchemaPlus rootSchema) {
                 rootSchema.add("hr",
                     new ReflectiveSchema(new JdbcTest.HrSchema()));
-                TableAccessRel table =
-                    new TableAccessRel(cluster,
+                LogicalTableScan table =
+                    new LogicalTableScan(cluster,
                         relOptSchema.getTableForMember(
                             Arrays.asList("hr", "emps")));
                 final RexBuilder rexBuilder = cluster.getRexBuilder();
-                FilterRel filter =
-                    new FilterRel(cluster, table,
+                LogicalFilter filter =
+                    new LogicalFilter(cluster, table,
                         rexBuilder.makeCall(
                             SqlStdOperatorTable.EQUALS,
                             rexBuilder.makeFieldAccess(
@@ -133,8 +137,8 @@ public class RelWriterTest {
                     cluster.getTypeFactory().createSqlType(SqlTypeName.INTEGER);
                 final RelDataType bigIntType =
                     cluster.getTypeFactory().createSqlType(SqlTypeName.BIGINT);
-                AggregateRel aggregate =
-                    new AggregateRel(cluster, filter, BitSets.of(0),
+                LogicalAggregate aggregate =
+                    new LogicalAggregate(cluster, filter, BitSets.of(0),
                         ImmutableList.of(
                             new AggregateCall(SqlStdOperatorTable.COUNT,
                                 true, ImmutableList.of(1), bigIntType, "c"),
@@ -149,7 +153,7 @@ public class RelWriterTest {
   }
 
   /**
-   * Unit test for {@link org.eigenbase.rel.RelJsonReader}.
+   * Unit test for {@link org.apache.calcite.rel.externalize.RelJsonReader}.
    */
   @Test public void testReader() {
     String s =
@@ -177,9 +181,9 @@ public class RelWriterTest {
             });
 
     assertThat(Util.toLinux(s), is(
-        "AggregateRel(group=[{0}], agg#0=[COUNT(DISTINCT $1)], agg#1=[COUNT()])\n"
-        + "  FilterRel(condition=[=($1, 10)])\n"
-        + "    TableAccessRel(table=[[hr, emps]])\n"));
+        "LogicalAggregate(group=[{0}], agg#0=[COUNT(DISTINCT $1)], agg#1=[COUNT()])\n"
+            + "  LogicalFilter(condition=[=($1, 10)])\n"
+            + "    LogicalTableScan(table=[[hr, emps]])\n"));
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/plan/volcano/VolcanoPlannerTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/plan/volcano/VolcanoPlannerTest.java b/core/src/test/java/org/apache/calcite/plan/volcano/VolcanoPlannerTest.java
index 125fdb5..4d0a691 100644
--- a/core/src/test/java/org/apache/calcite/plan/volcano/VolcanoPlannerTest.java
+++ b/core/src/test/java/org/apache/calcite/plan/volcano/VolcanoPlannerTest.java
@@ -14,28 +14,51 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.relopt.volcano;
-
-import java.util.*;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.rel.convert.*;
-import org.eigenbase.rel.rules.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.rex.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.util.*;
-
-import net.hydromatic.optiq.rules.java.EnumerableConvention;
+package org.apache.calcite.plan.volcano;
+
+import org.apache.calcite.adapter.enumerable.EnumerableConvention;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.ConventionTraitDef;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptListener;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelOptQuery;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.AbstractRelNode;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelWriter;
+import org.apache.calcite.rel.SingleRel;
+import org.apache.calcite.rel.convert.ConverterImpl;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.logical.LogicalProject;
+import org.apache.calcite.rel.rules.ProjectRemoveRule;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.sql.type.SqlTypeFactoryImpl;
+import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
 
 import org.junit.Ignore;
 import org.junit.Test;
 
-import static org.hamcrest.CoreMatchers.*;
-import static org.junit.Assert.*;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
 
 /**
  * Unit test for {@link VolcanoPlanner the optimizer}.
@@ -203,7 +226,7 @@ public class VolcanoPlannerTest {
     planner.addRelTraitDef(ConventionTraitDef.INSTANCE);
 
     if (useRule) {
-      planner.addRule(RemoveTrivialProjectRule.INSTANCE);
+      planner.addRule(ProjectRemoveRule.INSTANCE);
     }
 
     planner.addRule(new PhysLeafRule());
@@ -458,6 +481,7 @@ public class VolcanoPlannerTest {
 
   //~ Inner Classes ----------------------------------------------------------
 
+  /** Leaf relational expression. */
   private abstract static class TestLeafRel extends AbstractRelNode {
     private String label;
 
@@ -492,6 +516,7 @@ public class VolcanoPlannerTest {
     }
   }
 
+  /** Relational expression with one input. */
   private abstract static class TestSingleRel extends SingleRel {
     protected TestSingleRel(
         RelOptCluster cluster,
@@ -507,10 +532,11 @@ public class VolcanoPlannerTest {
 
     // implement RelNode
     protected RelDataType deriveRowType() {
-      return getChild().getRowType();
+      return getInput().getRowType();
     }
   }
 
+  /** Relational expression with one input and convention NONE. */
   private static class NoneSingleRel extends TestSingleRel {
     protected NoneSingleRel(
         RelOptCluster cluster,
@@ -529,6 +555,7 @@ public class VolcanoPlannerTest {
     }
   }
 
+  /** Relational expression with zero inputs and convention NONE. */
   private static class NoneLeafRel extends TestLeafRel {
     protected NoneLeafRel(
         RelOptCluster cluster,
@@ -546,6 +573,7 @@ public class VolcanoPlannerTest {
     }
   }
 
+  /** Relational expression with zero inputs and convention PHYS. */
   private static class PhysLeafRel extends TestLeafRel {
     PhysLeafRel(
         RelOptCluster cluster,
@@ -568,6 +596,7 @@ public class VolcanoPlannerTest {
     }
   }
 
+  /** Relational expression with one input and convention PHYS. */
   private static class PhysSingleRel extends TestSingleRel {
     PhysSingleRel(
         RelOptCluster cluster,
@@ -591,7 +620,8 @@ public class VolcanoPlannerTest {
     }
   }
 
-  class PhysToIteratorConverter extends ConverterRelImpl {
+  /** Converter from PHYS to ENUMERABLE convention. */
+  class PhysToIteratorConverter extends ConverterImpl {
     public PhysToIteratorConverter(
         RelOptCluster cluster,
         RelNode child) {
@@ -610,6 +640,8 @@ public class VolcanoPlannerTest {
     }
   }
 
+  /** Planner rule that converts {@link NoneLeafRel} to PHYS
+   * convention. */
   private static class PhysLeafRule extends RelOptRule {
     PhysLeafRule() {
       super(operand(NoneLeafRel.class, any()));
@@ -630,6 +662,7 @@ public class VolcanoPlannerTest {
     }
   }
 
+  /** Planner rule that matches a {@link NoneSingleRel} and succeeds. */
   private static class GoodSingleRule extends RelOptRule {
     GoodSingleRule() {
       super(operand(NoneSingleRel.class, any()));
@@ -643,7 +676,7 @@ public class VolcanoPlannerTest {
     // implement RelOptRule
     public void onMatch(RelOptRuleCall call) {
       NoneSingleRel singleRel = call.rel(0);
-      RelNode childRel = singleRel.getChild();
+      RelNode childRel = singleRel.getInput();
       RelNode physInput =
           convert(
               childRel,
@@ -655,6 +688,7 @@ public class VolcanoPlannerTest {
     }
   }
 
+  /** Rule that matches a {@link RelSubset}. */
   private static class SubsetRule extends RelOptRule {
     private final List<String> buf;
 
@@ -683,6 +717,9 @@ public class VolcanoPlannerTest {
   // ReformedSingleRule never saw it.  (GoodSingleRule saw the NoneLeafRel
   // instead and fires off of that; later the NoneLeafRel gets converted into
   // a PhysLeafRel).  Now Volcano supports rules which match across subsets.
+
+  /** Planner rule that matches a {@link NoneSingleRel} whose input is
+   * a {@link PhysLeafRel} in a different subset. */
   private static class ReformedSingleRule extends RelOptRule {
     ReformedSingleRule() {
       super(
@@ -711,9 +748,10 @@ public class VolcanoPlannerTest {
     }
   }
 
+  /** Planner rule that converts a {@link LogicalProject} to PHYS convention. */
   private static class PhysProjectRule extends RelOptRule {
     PhysProjectRule() {
-      super(operand(ProjectRel.class, any()));
+      super(operand(LogicalProject.class, any()));
     }
 
     // implement RelOptRule
@@ -723,8 +761,8 @@ public class VolcanoPlannerTest {
 
     // implement RelOptRule
     public void onMatch(RelOptRuleCall call) {
-      final ProjectRel project = call.rel(0);
-      RelNode childRel = project.getChild();
+      final LogicalProject project = call.rel(0);
+      RelNode childRel = project.getInput();
       call.transformTo(
           new PhysLeafRel(
               childRel.getCluster(),
@@ -732,6 +770,7 @@ public class VolcanoPlannerTest {
     }
   }
 
+  /** Planner rule that successfully removes a {@link PhysSingleRel}. */
   private static class GoodRemoveSingleRule extends RelOptRule {
     GoodRemoveSingleRule() {
       super(
@@ -756,6 +795,7 @@ public class VolcanoPlannerTest {
     }
   }
 
+  /** Planner rule that removes a {@link NoneSingleRel}. */
   private static class ReformedRemoveSingleRule extends RelOptRule {
     ReformedRemoveSingleRule() {
       super(
@@ -780,6 +820,7 @@ public class VolcanoPlannerTest {
     }
   }
 
+  /** Implementation of {@link RelOptListener}. */
   private static class TestListener implements RelOptListener {
     private List<RelEvent> eventList;
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/plan/volcano/VolcanoPlannerTraitTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/plan/volcano/VolcanoPlannerTraitTest.java b/core/src/test/java/org/apache/calcite/plan/volcano/VolcanoPlannerTraitTest.java
index 230a0bb..aefed30 100644
--- a/core/src/test/java/org/apache/calcite/plan/volcano/VolcanoPlannerTraitTest.java
+++ b/core/src/test/java/org/apache/calcite/plan/volcano/VolcanoPlannerTraitTest.java
@@ -14,17 +14,29 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.relopt.volcano;
-
-import java.util.List;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.rel.convert.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.util.*;
-
-import net.hydromatic.optiq.rules.java.EnumerableConvention;
+package org.apache.calcite.plan.volcano;
+
+import org.apache.calcite.adapter.enumerable.EnumerableConvention;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.ConventionTraitDef;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.RelTrait;
+import org.apache.calcite.plan.RelTraitDef;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.AbstractRelNode;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelWriter;
+import org.apache.calcite.rel.SingleRel;
+import org.apache.calcite.rel.convert.ConverterImpl;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.util.Pair;
 
 import com.google.common.collect.HashMultimap;
 import com.google.common.collect.Multimap;
@@ -32,7 +44,10 @@ import com.google.common.collect.Multimap;
 import org.junit.Ignore;
 import org.junit.Test;
 
-import static org.junit.Assert.*;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 /**
  * Unit test for handling of traits by {@link VolcanoPlanner}.
@@ -103,8 +118,7 @@ public class VolcanoPlannerTraitTest {
 
     NoneSingleRel noneRel =
         RelOptUtil.addTrait(
-            new NoneSingleRel(cluster, noneLeafRel),
-            ALT_TRAIT2);
+            new NoneSingleRel(cluster, noneLeafRel), ALT_TRAIT2);
 
     RelNode convertedRel =
         planner.changeTraits(
@@ -156,13 +170,11 @@ public class VolcanoPlannerTraitTest {
 
     NoneLeafRel noneLeafRel =
         RelOptUtil.addTrait(
-            new NoneLeafRel(cluster, "noneLeafRel"),
-            ALT_TRAIT);
+            new NoneLeafRel(cluster, "noneLeafRel"), ALT_TRAIT);
 
     NoneSingleRel noneRel =
         RelOptUtil.addTrait(
-            new NoneSingleRel(cluster, noneLeafRel),
-            ALT_TRAIT2);
+            new NoneSingleRel(cluster, noneLeafRel), ALT_TRAIT2);
 
     RelNode convertedRel =
         planner.changeTraits(
@@ -206,6 +218,7 @@ public class VolcanoPlannerTraitTest {
 
   //~ Inner Classes ----------------------------------------------------------
 
+  /** Implementation of {@link RelTrait} for testing. */
   private static class AltTrait implements RelTrait {
     private final AltTraitDef traitDef;
     private final int ordinal;
@@ -247,6 +260,7 @@ public class VolcanoPlannerTraitTest {
     }
   }
 
+  /** Definition of {@link AltTrait}. */
   private static class AltTraitDef extends RelTraitDef<AltTrait> {
     private Multimap<RelTrait, Pair<RelTrait, ConverterRule>> conversionMap =
         HashMultimap.create();
@@ -321,6 +335,7 @@ public class VolcanoPlannerTraitTest {
     }
   }
 
+  /** A relational expression with zero inputs. */
   private abstract static class TestLeafRel extends AbstractRelNode {
     private String label;
 
@@ -355,6 +370,7 @@ public class VolcanoPlannerTraitTest {
     }
   }
 
+  /** A relational expression with zero inputs, of NONE convention. */
   private static class NoneLeafRel extends TestLeafRel {
     protected NoneLeafRel(
         RelOptCluster cluster,
@@ -365,12 +381,12 @@ public class VolcanoPlannerTraitTest {
           label);
     }
 
-    @Override
-    public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+    @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
       return new NoneLeafRel(getCluster(), getLabel());
     }
   }
 
+  /** Relational expression with zero inputs, of PHYS convention. */
   private static class PhysLeafRel extends TestLeafRel {
     PhysLeafRel(
         RelOptCluster cluster,
@@ -389,6 +405,7 @@ public class VolcanoPlannerTraitTest {
     // TODO: SWZ Implement clone?
   }
 
+  /** Relational expression with one input. */
   private abstract static class TestSingleRel extends SingleRel {
     protected TestSingleRel(
         RelOptCluster cluster,
@@ -404,12 +421,13 @@ public class VolcanoPlannerTraitTest {
 
     // implement RelNode
     protected RelDataType deriveRowType() {
-      return getChild().getRowType();
+      return getInput().getRowType();
     }
 
     // TODO: SWZ Implement clone?
   }
 
+  /** Relational expression with one input, of NONE convention. */
   private static class NoneSingleRel extends TestSingleRel {
     protected NoneSingleRel(
         RelOptCluster cluster,
@@ -436,13 +454,17 @@ public class VolcanoPlannerTraitTest {
   }
 
 
+  /** A mix-in interface to extend {@link RelNode}, for testing. */
   interface FooRel {
     String implement(FooRelImplementor implementor);
   }
 
+  /** An implementor for {@link FooRel}. */
   interface FooRelImplementor {
   }
 
+  /** Relational expression with one input, that implements the {@link FooRel}
+   * mix-in interface. */
   private static class IterSingleRel extends TestSingleRel implements FooRel {
     public IterSingleRel(RelOptCluster cluster, RelNode child) {
       super(
@@ -468,6 +490,7 @@ public class VolcanoPlannerTraitTest {
     }
   }
 
+  /** Relational expression with zero inputs, of the PHYS convention. */
   private static class PhysLeafRule extends RelOptRule {
     PhysLeafRule() {
       super(operand(NoneLeafRel.class, any()));
@@ -488,6 +511,8 @@ public class VolcanoPlannerTraitTest {
     }
   }
 
+  /** Planner rule to convert a {@link NoneSingleRel} to ENUMERABLE
+   * convention. */
   private static class IterSingleRule extends RelOptRule {
     IterSingleRule() {
       super(operand(NoneSingleRel.class, any()));
@@ -518,6 +543,8 @@ public class VolcanoPlannerTraitTest {
     }
   }
 
+  /** Another planner rule to convert a {@link NoneSingleRel} to ENUMERABLE
+   * convention. */
   private static class IterSingleRule2 extends RelOptRule {
     IterSingleRule2() {
       super(operand(NoneSingleRel.class, any()));
@@ -553,6 +580,7 @@ public class VolcanoPlannerTraitTest {
     }
   }
 
+  /** Planner rule that converts between {@link AltTrait}s. */
   private static class AltTraitConverterRule extends ConverterRule {
     private final RelTrait toTrait;
 
@@ -581,7 +609,8 @@ public class VolcanoPlannerTraitTest {
     }
   }
 
-  private static class AltTraitConverter extends ConverterRelImpl {
+  /** Relational expression that converts between {@link AltTrait} values. */
+  private static class AltTraitConverter extends ConverterImpl {
     private final RelTrait toTrait;
 
     private AltTraitConverter(
@@ -605,6 +634,7 @@ public class VolcanoPlannerTraitTest {
     }
   }
 
+  /** Planner rule that converts from PHYS to ENUMERABLE convention. */
   private static class PhysToIteratorConverterRule extends ConverterRule {
     public PhysToIteratorConverterRule() {
       super(
@@ -621,7 +651,8 @@ public class VolcanoPlannerTraitTest {
     }
   }
 
-  private static class PhysToIteratorConverter extends ConverterRelImpl {
+  /** Planner rule that converts PHYS to ENUMERABLE convention. */
+  private static class PhysToIteratorConverter extends ConverterImpl {
     public PhysToIteratorConverter(
         RelOptCluster cluster,
         RelNode child) {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/rex/RexExecutorTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/rex/RexExecutorTest.java b/core/src/test/java/org/apache/calcite/rex/RexExecutorTest.java
index 27c36ad..b56c928 100644
--- a/core/src/test/java/org/apache/calcite/rex/RexExecutorTest.java
+++ b/core/src/test/java/org/apache/calcite/rex/RexExecutorTest.java
@@ -14,40 +14,39 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rex;
-
-import java.math.BigDecimal;
-import java.util.*;
-
-import org.eigenbase.relopt.RelOptCluster;
-import org.eigenbase.relopt.RelOptSchema;
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.reltype.RelDataTypeFactory;
-import org.eigenbase.sql.fun.SqlStdOperatorTable;
-import org.eigenbase.sql.type.SqlTypeName;
-import org.eigenbase.util.NlsString;
-
-import net.hydromatic.linq4j.QueryProvider;
-
-import net.hydromatic.optiq.DataContext;
-import net.hydromatic.optiq.SchemaPlus;
-import net.hydromatic.optiq.Schemas;
-import net.hydromatic.optiq.impl.java.JavaTypeFactory;
-import net.hydromatic.optiq.server.OptiqServerStatement;
-import net.hydromatic.optiq.tools.Frameworks;
+package org.apache.calcite.rex;
+
+import org.apache.calcite.DataContext;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.linq4j.QueryProvider;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptSchema;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.schema.Schemas;
+import org.apache.calcite.server.CalciteServerStatement;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.tools.Frameworks;
+import org.apache.calcite.util.NlsString;
 
 import com.google.common.collect.ImmutableList;
 
 import org.junit.Assert;
 import org.junit.Test;
 
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.List;
+
 import static org.hamcrest.CoreMatchers.equalTo;
 import static org.hamcrest.CoreMatchers.instanceOf;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 
 /**
- * Unit test for {@link org.eigenbase.rex.RexExecutorImpl}.
+ * Unit test for {@link org.apache.calcite.rex.RexExecutorImpl}.
  */
 public class RexExecutorTest {
   public RexExecutorTest() {
@@ -57,7 +56,7 @@ public class RexExecutorTest {
     Frameworks.withPrepare(
         new Frameworks.PrepareAction<Void>() {
           public Void apply(RelOptCluster cluster, RelOptSchema relOptSchema,
-              SchemaPlus rootSchema, OptiqServerStatement statement) {
+              SchemaPlus rootSchema, CalciteServerStatement statement) {
             final RexBuilder rexBuilder = cluster.getRexBuilder();
             DataContext dataContext =
                 Schemas.createDataContext(statement.getConnection());
@@ -154,7 +153,7 @@ public class RexExecutorTest {
 
   /** Callback for {@link #check}. Test code will typically use {@code builder}
    * to create some expressions, call
-   * {@link org.eigenbase.rex.RexExecutorImpl#reduce} to evaluate them into
+   * {@link org.apache.calcite.rex.RexExecutorImpl#reduce} to evaluate them into
    * a list, then check that the results are as expected. */
   interface Action {
     void check(RexBuilder rexBuilder, RexExecutorImpl executor);

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/runtime/BinarySearchTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/runtime/BinarySearchTest.java b/core/src/test/java/org/apache/calcite/runtime/BinarySearchTest.java
index 9410ecd..345a324 100644
--- a/core/src/test/java/org/apache/calcite/runtime/BinarySearchTest.java
+++ b/core/src/test/java/org/apache/calcite/runtime/BinarySearchTest.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.runtime;
+package org.apache.calcite.runtime;
 
 import com.google.common.collect.Ordering;
 
@@ -24,7 +24,7 @@ import org.junit.Test;
 import java.util.Arrays;
 
 /**
- * Tests {@link net.hydromatic.optiq.runtime.BinarySearch}.
+ * Tests {@link org.apache.calcite.runtime.BinarySearch}.
  */
 public class BinarySearchTest {
   private void search(int key, int lower, int upper, Integer... array) {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/runtime/EnumerablesTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/runtime/EnumerablesTest.java b/core/src/test/java/org/apache/calcite/runtime/EnumerablesTest.java
index f68451c..a8cb464 100644
--- a/core/src/test/java/org/apache/calcite/runtime/EnumerablesTest.java
+++ b/core/src/test/java/org/apache/calcite/runtime/EnumerablesTest.java
@@ -14,11 +14,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.runtime;
+package org.apache.calcite.runtime;
 
-import net.hydromatic.linq4j.Linq4j;
-import net.hydromatic.linq4j.function.Function1;
-import net.hydromatic.linq4j.function.Functions;
+import org.apache.calcite.linq4j.Linq4j;
+import org.apache.calcite.linq4j.function.Function1;
+import org.apache.calcite.linq4j.function.Functions;
 
 import org.junit.Test;
 
@@ -28,7 +28,7 @@ import static org.hamcrest.CoreMatchers.equalTo;
 import static org.junit.Assert.assertThat;
 
 /**
- * Unit tests for {@link net.hydromatic.optiq.runtime.Enumerables}.
+ * Unit tests for {@link org.apache.calcite.runtime.Enumerables}.
  */
 public class EnumerablesTest {
   @Test public void testSemiJoin() {


[14/58] [abbrv] [partial] incubator-calcite git commit: [CALCITE-306] Standardize code style for "import package.*; "

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/test/LatticeTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/LatticeTest.java b/core/src/test/java/org/apache/calcite/test/LatticeTest.java
index e9c1ac4..e70b80b 100644
--- a/core/src/test/java/org/apache/calcite/test/LatticeTest.java
+++ b/core/src/test/java/org/apache/calcite/test/LatticeTest.java
@@ -14,15 +14,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.test;
+package org.apache.calcite.test;
 
-import net.hydromatic.optiq.materialize.MaterializationService;
-import net.hydromatic.optiq.runtime.Hook;
-
-import org.eigenbase.rel.RelNode;
-import org.eigenbase.relopt.RelOptUtil;
-import org.eigenbase.util.TestUtil;
-import org.eigenbase.util.Util;
+import org.apache.calcite.materialize.MaterializationService;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.runtime.Hook;
+import org.apache.calcite.util.TestUtil;
+import org.apache.calcite.util.Util;
 
 import com.google.common.base.Function;
 import com.google.common.collect.ImmutableList;
@@ -49,7 +48,7 @@ import static org.junit.Assert.assertThat;
  * Unit test for lattices.
  */
 public class LatticeTest {
-  private OptiqAssert.AssertThat modelWithLattice(String name, String sql,
+  private CalciteAssert.AssertThat modelWithLattice(String name, String sql,
       String... extras) {
     final StringBuilder buf = new StringBuilder("{ name: '")
         .append(name)
@@ -62,10 +61,10 @@ public class LatticeTest {
     return modelWithLattices(buf.toString());
   }
 
-  private OptiqAssert.AssertThat modelWithLattices(String... lattices) {
+  private CalciteAssert.AssertThat modelWithLattices(String... lattices) {
     final Class<JdbcTest.EmpDeptTableFactory> clazz =
         JdbcTest.EmpDeptTableFactory.class;
-    return OptiqAssert.that().withModel(""
+    return CalciteAssert.that().withModel(""
         + "{\n"
         + "  version: '1.0',\n"
         + "   schemas: [\n"
@@ -118,14 +117,14 @@ public class LatticeTest {
   @Test public void testLatticeSqlWithGroupByFails() {
     modelWithLattice("star",
         "select 1 from \"foodmart\".\"sales_fact_1997\" as s group by \"product_id\"")
-        .connectThrows("Invalid node type AggregateRel in lattice query");
+        .connectThrows("Invalid node type LogicalAggregate in lattice query");
   }
 
   /** Tests a lattice whose SQL is invalid because it contains a ORDER BY. */
   @Test public void testLatticeSqlWithOrderByFails() {
     modelWithLattice("star",
         "select 1 from \"foodmart\".\"sales_fact_1997\" as s order by \"product_id\"")
-        .connectThrows("Invalid node type SortRel in lattice query");
+        .connectThrows("Invalid node type Sort in lattice query");
   }
 
   /** Tests a lattice whose SQL is invalid because it contains a UNION ALL. */
@@ -134,7 +133,7 @@ public class LatticeTest {
         "select 1 from \"foodmart\".\"sales_fact_1997\" as s\n"
         + "union all\n"
         + "select 1 from \"foodmart\".\"sales_fact_1997\" as s")
-        .connectThrows("Invalid node type UnionRel in lattice query");
+        .connectThrows("Invalid node type LogicalUnion in lattice query");
   }
 
   /** Tests a lattice with valid join SQL. */
@@ -170,10 +169,10 @@ public class LatticeTest {
       foodmartModel()
           .query("select count(*) from \"adhoc\".\"star\"")
           .convertMatches(
-              OptiqAssert.checkRel(
-                  "AggregateRel(group=[{}], EXPR$0=[COUNT()])\n"
-                  + "  ProjectRel(DUMMY=[0])\n"
-                  + "    StarTableScan(table=[[adhoc, star]])\n",
+              CalciteAssert.checkRel(
+                  "LogicalAggregate(group=[{}], EXPR$0=[COUNT()])\n"
+                      + "  LogicalProject(DUMMY=[0])\n"
+                      + "    StarTableScan(table=[[adhoc, star]])\n",
                   counter));
     } catch (RuntimeException e) {
       assertThat(Util.getStackTrace(e), containsString("CannotPlanException"));
@@ -185,16 +184,15 @@ public class LatticeTest {
   @Test public void testLatticeRecognizeJoin() {
     final AtomicInteger counter = new AtomicInteger();
     foodmartModel()
-        .query(
-            "select s.\"unit_sales\", p.\"brand_name\"\n"
+        .query("select s.\"unit_sales\", p.\"brand_name\"\n"
             + "from \"foodmart\".\"sales_fact_1997\" as s\n"
             + "join \"foodmart\".\"product\" as p using (\"product_id\")\n")
         .enableMaterializations(true)
         .substitutionMatches(
-            OptiqAssert.checkRel(
-                "ProjectRel(unit_sales=[$7], brand_name=[$10])\n"
-                + "  ProjectRel($f0=[$0], $f1=[$1], $f2=[$2], $f3=[$3], $f4=[$4], $f5=[$5], $f6=[$6], $f7=[$7], $f8=[$8], $f9=[$9], $f10=[$10], $f11=[$11], $f12=[$12], $f13=[$13], $f14=[$14], $f15=[$15], $f16=[$16], $f17=[$17], $f18=[$18], $f19=[$19], $f20=[$20], $f21=[$21], $f22=[$22])\n"
-                + "    TableAccessRel(table=[[adhoc, star]])\n",
+            CalciteAssert.checkRel(
+                "LogicalProject(unit_sales=[$7], brand_name=[$10])\n"
+                    + "  LogicalProject($f0=[$0], $f1=[$1], $f2=[$2], $f3=[$3], $f4=[$4], $f5=[$5], $f6=[$6], $f7=[$7], $f8=[$8], $f9=[$9], $f10=[$10], $f11=[$11], $f12=[$12], $f13=[$13], $f14=[$14], $f15=[$15], $f16=[$16], $f17=[$17], $f18=[$18], $f19=[$19], $f20=[$20], $f21=[$21], $f22=[$22])\n"
+                    + "    LogicalTableScan(table=[[adhoc, star]])\n",
                 counter));
     assertThat(counter.intValue(), equalTo(1));
   }
@@ -202,9 +200,8 @@ public class LatticeTest {
   /** Tests an aggregate on a 2-way join query can use an aggregate table. */
   @Test public void testLatticeRecognizeGroupJoin() {
     final AtomicInteger counter = new AtomicInteger();
-    OptiqAssert.AssertQuery that = foodmartModel()
-        .query(
-            "select distinct p.\"brand_name\", s.\"customer_id\"\n"
+    CalciteAssert.AssertQuery that = foodmartModel()
+        .query("select distinct p.\"brand_name\", s.\"customer_id\"\n"
             + "from \"foodmart\".\"sales_fact_1997\" as s\n"
             + "join \"foodmart\".\"product\" as p using (\"product_id\")\n")
         .enableMaterializations(true)
@@ -216,19 +213,19 @@ public class LatticeTest {
                 assertThat(s,
                     anyOf(
                         containsString(
-                            "ProjectRel($f0=[$1], $f1=[$0])\n"
-                            + "  AggregateRel(group=[{2, 10}])\n"
-                            + "    TableAccessRel(table=[[adhoc, star]])\n"),
+                            "LogicalProject($f0=[$1], $f1=[$0])\n"
+                            + "  LogicalAggregate(group=[{2, 10}])\n"
+                            + "    LogicalTableScan(table=[[adhoc, star]])\n"),
                         containsString(
-                            "AggregateRel(group=[{2, 10}])\n"
-                            + "  TableAccessRel(table=[[adhoc, star]])\n")));
+                            "LogicalAggregate(group=[{2, 10}])\n"
+                            + "  LogicalTableScan(table=[[adhoc, star]])\n")));
                 return null;
               }
             });
     assertThat(counter.intValue(), equalTo(2));
-    that.explainContains(
-        "EnumerableCalcRel(expr#0..1=[{inputs}], $f0=[$t1], $f1=[$t0])\n"
-        + "  EnumerableTableAccessRel(table=[[adhoc, m{2, 10}]])")
+    that.explainContains(""
+        + "EnumerableCalc(expr#0..1=[{inputs}], $f0=[$t1], $f1=[$t0])\n"
+        + "  EnumerableTableScan(table=[[adhoc, m{2, 10}]])")
         .returnsCount(69203);
 
     // Run the same query again and see whether it uses the same
@@ -244,7 +241,7 @@ public class LatticeTest {
         .returnsCount(69203);
 
     // Ideally the counter would stay at 2. It increments to 3 because
-    // OptiqAssert.AssertQuery creates a new schema for every request,
+    // CalciteAssert.AssertQuery creates a new schema for every request,
     // and therefore cannot re-use lattices or materializations from the
     // previous request.
     assertThat(counter.intValue(), equalTo(3));
@@ -264,8 +261,7 @@ public class LatticeTest {
             + "from \"foodmart\".\"sales_fact_1997\" as s\n"
             + "join \"foodmart\".\"time_by_day\" as t using (\"time_id\")\n")
       .enableMaterializations(true)
-      .explainContains(
-          "EnumerableTableAccessRel(table=[[adhoc, m{27, 31}")
+      .explainContains("EnumerableTableScan(table=[[adhoc, m{27, 31}")
       .returnsCount(4);
   }
 
@@ -278,9 +274,9 @@ public class LatticeTest {
             + "join \"foodmart\".\"time_by_day\" as t using (\"time_id\")\n"
             + "group by t.\"the_year\", t.\"quarter\"")
       .enableMaterializations(true)
-      .explainContains(
-          "EnumerableCalcRel(expr#0..4=[{inputs}], proj#0..2=[{exprs}])\n"
-          + "  EnumerableTableAccessRel(table=[[adhoc, m{27, 31}")
+      .explainContains(""
+          + "EnumerableCalc(expr#0..4=[{inputs}], proj#0..2=[{exprs}])\n"
+          + "  EnumerableTableScan(table=[[adhoc, m{27, 31}")
       .returnsUnordered("the_year=1997; quarter=Q1; C=21588",
           "the_year=1997; quarter=Q2; C=20368",
           "the_year=1997; quarter=Q3; C=21453",
@@ -301,10 +297,10 @@ public class LatticeTest {
             + "join \"foodmart\".\"time_by_day\" as t using (\"time_id\")\n"
             + "group by t.\"the_year\"")
       .enableMaterializations(true)
-      .explainContains(
-          "EnumerableCalcRel(expr#0..3=[{inputs}], expr#4=[10], expr#5=[*($t3, $t4)], proj#0..2=[{exprs}], US=[$t5])\n"
-          + "  EnumerableAggregateRel(group=[{0}], C=[$SUM0($2)], Q=[MIN($1)], agg#2=[$SUM0($4)])\n"
-          + "    EnumerableTableAccessRel(table=[[adhoc, m{27, 31}")
+      .explainContains(""
+          + "EnumerableCalc(expr#0..3=[{inputs}], expr#4=[10], expr#5=[*($t3, $t4)], proj#0..2=[{exprs}], US=[$t5])\n"
+          + "  EnumerableAggregate(group=[{0}], C=[$SUM0($2)], Q=[MIN($1)], agg#2=[$SUM0($4)])\n"
+          + "    EnumerableTableScan(table=[[adhoc, m{27, 31}")
       .returnsUnordered("the_year=1997; C=86837; Q=Q1; US=2667730.0000")
       .sameResultWithMaterializationsDisabled();
   }
@@ -337,13 +333,12 @@ public class LatticeTest {
         + "    dimensions: [ 'the_year', ['t', 'quarter'] ],\n"
         + "    measures: [ ]\n"
         + "  } ]\n")
-        .query(
-            "select distinct t.\"the_year\", t.\"quarter\"\n"
+        .query("select distinct t.\"the_year\", t.\"quarter\"\n"
             + "from \"foodmart\".\"sales_fact_1997\" as s\n"
             + "join \"foodmart\".\"time_by_day\" as t using (\"time_id\")\n")
         .enableMaterializations(true)
-        .explainContains("EnumerableAggregateRel(group=[{2, 3}])\n"
-            + "  EnumerableTableAccessRel(table=[[adhoc, m{16, 17, 27, 31}]])")
+        .explainContains("EnumerableAggregate(group=[{2, 3}])\n"
+            + "  EnumerableTableScan(table=[[adhoc, m{16, 17, 27, 31}]])")
         .returnsUnordered("the_year=1997; quarter=Q1",
             "the_year=1997; quarter=Q2",
             "the_year=1997; quarter=Q3",
@@ -386,12 +381,12 @@ public class LatticeTest {
             return null;
           }
         };
-    final OptiqAssert.AssertThat that = foodmartModel().pooled();
+    final CalciteAssert.AssertThat that = foodmartModel().pooled();
     that.query("select sum(\"unit_sales\") as s, count(*) as c\n"
             + "from \"foodmart\".\"sales_fact_1997\"")
         .withHook(Hook.CREATE_MATERIALIZATION, handler)
         .enableMaterializations(true)
-        .explainContains("EnumerableTableAccessRel(table=[[adhoc, m{}]])")
+        .explainContains("EnumerableTableScan(table=[[adhoc, m{}]])")
         .returnsUnordered("S=266773.0000; C=86837");
     assertThat(mats.toString(), mats.size(), equalTo(2));
 
@@ -426,10 +421,10 @@ public class LatticeTest {
             + "join \"foodmart\".\"time_by_day\" using (\"time_id\")\n"
             + "group by \"the_year\"")
         .enableMaterializations(true)
-        .explainContains("EnumerableCalcRel(expr#0..1=[{inputs}], C=[$t1])\n"
-            + "  EnumerableAggregateRel(group=[{0}], C=[COUNT($1)])\n"
-            + "    EnumerableCalcRel(expr#0..4=[{inputs}], proj#0..1=[{exprs}])\n"
-            + "      EnumerableTableAccessRel(table=[[adhoc, m{27, 31}]])")
+        .explainContains("EnumerableCalc(expr#0..1=[{inputs}], C=[$t1])\n"
+            + "  EnumerableAggregate(group=[{0}], C=[COUNT($1)])\n"
+            + "    EnumerableCalc(expr#0..4=[{inputs}], proj#0..1=[{exprs}])\n"
+            + "      EnumerableTableScan(table=[[adhoc, m{27, 31}]])")
         .returnsUnordered("C=4");
   }
 
@@ -440,10 +435,10 @@ public class LatticeTest {
             + "join \"foodmart\".\"time_by_day\" using (\"time_id\")\n"
             + "group by \"the_year\"")
         .enableMaterializations(true)
-        .explainContains("EnumerableCalcRel(expr#0..1=[{inputs}], C=[$t1])\n"
-            + "  EnumerableAggregateRel(group=[{0}], C=[COUNT($0)])\n"
-            + "    EnumerableAggregateRel(group=[{0}])\n"
-            + "      EnumerableTableAccessRel(table=[[adhoc, m{27, 31}]])")
+        .explainContains("EnumerableCalc(expr#0..1=[{inputs}], C=[$t1])\n"
+            + "  EnumerableAggregate(group=[{0}], C=[COUNT($0)])\n"
+            + "    EnumerableAggregate(group=[{0}])\n"
+            + "      EnumerableTableScan(table=[[adhoc, m{27, 31}]])")
         .returnsUnordered("C=1");
   }
 
@@ -512,18 +507,17 @@ public class LatticeTest {
     // TODO
   }
 
-  private OptiqAssert.AssertThat foodmartModel(String... extras) {
+  private CalciteAssert.AssertThat foodmartModel(String... extras) {
     return modelWithLattice("star",
         "select 1 from \"foodmart\".\"sales_fact_1997\" as \"s\"\n"
-        + "join \"foodmart\".\"product\" as \"p\" using (\"product_id\")\n"
-        + "join \"foodmart\".\"time_by_day\" as \"t\" using (\"time_id\")\n"
-        + "join \"foodmart\".\"product_class\" as \"pc\" on \"p\".\"product_class_id\" = \"pc\".\"product_class_id\"",
+            + "join \"foodmart\".\"product\" as \"p\" using (\"product_id\")\n"
+            + "join \"foodmart\".\"time_by_day\" as \"t\" using (\"time_id\")\n"
+            + "join \"foodmart\".\"product_class\" as \"pc\" on \"p\".\"product_class_id\" = \"pc\".\"product_class_id\"",
         extras);
   }
 
-  private OptiqAssert.AssertThat foodmartModelWithOneTile() {
-    return foodmartModel(
-        " auto: false,\n"
+  private CalciteAssert.AssertThat foodmartModelWithOneTile() {
+    return foodmartModel(" auto: false,\n"
         + "  defaultMeasures: [ {\n"
         + "    agg: 'count'\n"
         + "  } ],\n"
@@ -547,7 +541,7 @@ public class LatticeTest {
         "jdbc:calcite:model=core/src/test/resources/mysql-foodmart-lattice-model.json");
     final ResultSet resultSet = connection.createStatement()
         .executeQuery("select * from \"adhoc\".\"m{27, 31}\"");
-    System.out.println(OptiqAssert.toString(resultSet));
+    System.out.println(CalciteAssert.toString(resultSet));
     connection.close();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/test/LinqFrontJdbcBackTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/LinqFrontJdbcBackTest.java b/core/src/test/java/org/apache/calcite/test/LinqFrontJdbcBackTest.java
index a7218e2..5349579 100644
--- a/core/src/test/java/org/apache/calcite/test/LinqFrontJdbcBackTest.java
+++ b/core/src/test/java/org/apache/calcite/test/LinqFrontJdbcBackTest.java
@@ -14,17 +14,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.test;
+package org.apache.calcite.test;
 
-import net.hydromatic.linq4j.expressions.Expressions;
-import net.hydromatic.linq4j.expressions.ParameterExpression;
-import net.hydromatic.linq4j.function.Predicate1;
-
-import net.hydromatic.optiq.SchemaPlus;
-import net.hydromatic.optiq.Schemas;
-import net.hydromatic.optiq.jdbc.OptiqConnection;
-
-import org.eigenbase.util.Util;
+import org.apache.calcite.jdbc.CalciteConnection;
+import org.apache.calcite.linq4j.function.Predicate1;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.linq4j.tree.ParameterExpression;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.schema.Schemas;
+import org.apache.calcite.util.Util;
 
 import org.junit.Test;
 
@@ -36,8 +34,8 @@ import java.sql.SQLException;
 public class LinqFrontJdbcBackTest {
   @Test public void testTableWhere() throws SQLException,
       ClassNotFoundException {
-    final OptiqConnection connection =
-        OptiqAssert.getConnection(OptiqAssert.SchemaSpec.JDBC_FOODMART);
+    final CalciteConnection connection =
+        CalciteAssert.getConnection(CalciteAssert.SchemaSpec.JDBC_FOODMART);
     final SchemaPlus schema =
         connection.getRootSchema().getSubSchema("foodmart");
     ParameterExpression c =

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/test/MaterializationTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/MaterializationTest.java b/core/src/test/java/org/apache/calcite/test/MaterializationTest.java
index 3b6827b..7fa1848 100644
--- a/core/src/test/java/org/apache/calcite/test/MaterializationTest.java
+++ b/core/src/test/java/org/apache/calcite/test/MaterializationTest.java
@@ -14,17 +14,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.test;
-
-import net.hydromatic.optiq.jdbc.JavaTypeFactoryImpl;
-import net.hydromatic.optiq.materialize.MaterializationService;
-import net.hydromatic.optiq.prepare.Prepare;
-
-import org.eigenbase.relopt.SubstitutionVisitor;
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.reltype.RelDataTypeSystem;
-import org.eigenbase.rex.*;
-import org.eigenbase.sql.fun.SqlStdOperatorTable;
+package org.apache.calcite.test;
+
+import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
+import org.apache.calcite.materialize.MaterializationService;
+import org.apache.calcite.plan.SubstitutionVisitor;
+import org.apache.calcite.prepare.Prepare;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 
 import com.google.common.base.Function;
 
@@ -34,8 +36,11 @@ import org.junit.Test;
 import java.math.BigDecimal;
 import java.sql.ResultSet;
 
-import static org.hamcrest.CoreMatchers.*;
-import static org.junit.Assert.*;
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
 
 /**
  * Unit test for the materialized view rewrite mechanism. Each test has a
@@ -44,16 +49,16 @@ import static org.junit.Assert.*;
  */
 public class MaterializationTest {
   private static final Function<ResultSet, Void> CONTAINS_M0 =
-      OptiqAssert.checkResultContains(
-          "EnumerableTableAccessRel(table=[[hr, m0]])");
+      CalciteAssert.checkResultContains(
+          "EnumerableTableScan(table=[[hr, m0]])");
 
   final JavaTypeFactoryImpl typeFactory =
       new JavaTypeFactoryImpl(RelDataTypeSystem.DEFAULT);
   final RexBuilder rexBuilder = new RexBuilder(typeFactory);
 
   @Test public void testFilter() {
-    OptiqAssert.that()
-        .with(OptiqAssert.Config.REGULAR)
+    CalciteAssert.that()
+        .with(CalciteAssert.Config.REGULAR)
         .withMaterializations(
             JdbcTest.HR_MODEL,
             "m0",
@@ -61,8 +66,7 @@ public class MaterializationTest {
         .query(
             "select \"empid\" + 1 from \"emps\" where \"deptno\" = 10")
         .enableMaterializations(true)
-        .explainContains(
-            "EnumerableTableAccessRel(table=[[hr, m0]])")
+        .explainContains("EnumerableTableScan(table=[[hr, m0]])")
         .sameResultWithMaterializationsDisabled();
   }
 
@@ -70,8 +74,8 @@ public class MaterializationTest {
     try {
       Prepare.THREAD_TRIM.set(true);
       MaterializationService.setThreadLocal();
-      OptiqAssert.that()
-          .with(OptiqAssert.Config.REGULAR)
+      CalciteAssert.that()
+          .with(CalciteAssert.Config.REGULAR)
           .withMaterializations(
               JdbcTest.HR_MODEL,
               "m0",
@@ -79,8 +83,7 @@ public class MaterializationTest {
           .query(
               "select \"empid\" + 1 as x from \"emps\" where \"deptno\" = 10")
           .enableMaterializations(true)
-          .explainContains(
-              "EnumerableTableAccessRel(table=[[hr, m0]])")
+          .explainContains("EnumerableTableScan(table=[[hr, m0]])")
           .sameResultWithMaterializationsDisabled();
     } finally {
       Prepare.THREAD_TRIM.set(false);
@@ -100,8 +103,8 @@ public class MaterializationTest {
     try {
       Prepare.THREAD_TRIM.set(true);
       MaterializationService.setThreadLocal();
-      OptiqAssert.that()
-          .with(OptiqAssert.Config.REGULAR)
+      CalciteAssert.that()
+          .with(CalciteAssert.Config.REGULAR)
           .withMaterializations(model, "m0", materialize)
           .query(query)
           .enableMaterializations(true)
@@ -119,13 +122,12 @@ public class MaterializationTest {
     try {
       Prepare.THREAD_TRIM.set(true);
       MaterializationService.setThreadLocal();
-      OptiqAssert.that()
-          .with(OptiqAssert.Config.REGULAR)
+      CalciteAssert.that()
+          .with(CalciteAssert.Config.REGULAR)
           .withMaterializations(model, "m0", materialize)
           .query(query)
           .enableMaterializations(true)
-          .explainContains(
-              "EnumerableTableAccessRel(table=[[hr, emps]])");
+          .explainContains("EnumerableTableScan(table=[[hr, emps]])");
     } finally {
       Prepare.THREAD_TRIM.set(false);
     }
@@ -179,9 +181,9 @@ public class MaterializationTest {
         "select \"name\", \"empid\" + 1 as e\n"
         + "from \"emps\" where \"deptno\" - 10 = 2",
         JdbcTest.HR_MODEL,
-        OptiqAssert.checkResultContains(
-            "EnumerableCalcRel(expr#0..2=[{inputs}], expr#3=[2], expr#4=[=($t0, $t3)], name=[$t2], E=[$t1], $condition=[$t4])\n"
-            + "  EnumerableTableAccessRel(table=[[hr, m0]]"));
+        CalciteAssert.checkResultContains(
+            "EnumerableCalc(expr#0..2=[{inputs}], expr#3=[2], expr#4=[=($t0, $t3)], name=[$t2], E=[$t1], $condition=[$t4])\n"
+                + "  EnumerableTableScan(table=[[hr, m0]]"));
   }
 
   /** Cannot materialize because "name" is not projected in the MV. */
@@ -224,9 +226,9 @@ public class MaterializationTest {
         "select \"deptno\", \"empid\", \"name\" from \"emps\" where \"deptno\" = 10 or \"deptno\" = 20 or \"empid\" < 160",
         "select \"empid\" + 1 as x, \"name\" from \"emps\" where \"deptno\" = 10",
         JdbcTest.HR_MODEL,
-        OptiqAssert.checkResultContains(
+        CalciteAssert.checkResultContains(
             "EnumerableCalcRel(expr#0..2=[{inputs}], expr#3=[1], expr#4=[+($t1, $t3)], X=[$t4], name=[$t2], condition=?)\n"
-            + "  EnumerableTableAccessRel(table=[[hr, m0]])"));
+                + "  EnumerableTableScan(table=[[hr, m0]])"));
   }
 
   /** Aggregation query at same level of aggregation as aggregation
@@ -238,17 +240,17 @@ public class MaterializationTest {
   }
 
   /** Aggregation query at coarser level of aggregation than aggregation
-   * materialization. Requires an additional AggregateRel to roll up. Note that
+   * materialization. Requires an additional aggregate to roll up. Note that
    * COUNT is rolled up using SUM. */
   @Test public void testAggregateRollUp() {
     checkMaterialize(
         "select \"empid\", \"deptno\", count(*) as c, sum(\"empid\") as s from \"emps\" group by \"empid\", \"deptno\"",
         "select count(*) + 1 as c, \"deptno\" from \"emps\" group by \"deptno\"",
         JdbcTest.HR_MODEL,
-        OptiqAssert.checkResultContains(
-            "EnumerableCalcRel(expr#0..1=[{inputs}], expr#2=[1], expr#3=[+($t1, $t2)], C=[$t3], deptno=[$t0])\n"
-            + "  EnumerableAggregateRel(group=[{1}], agg#0=[$SUM0($2)])\n"
-            + "    EnumerableTableAccessRel(table=[[hr, m0]])"));
+        CalciteAssert.checkResultContains(
+            "EnumerableCalc(expr#0..1=[{inputs}], expr#2=[1], expr#3=[+($t1, $t2)], C=[$t3], deptno=[$t0])\n"
+                + "  EnumerableAggregate(group=[{1}], agg#0=[$SUM0($2)])\n"
+                + "    EnumerableTableScan(table=[[hr, m0]])"));
   }
 
   /** Aggregation materialization with a project. */
@@ -260,7 +262,7 @@ public class MaterializationTest {
         "select \"deptno\", count(*) as c, \"empid\" + 2, sum(\"empid\") as s from \"emps\" group by \"empid\", \"deptno\"",
         "select count(*) + 1 as c, \"deptno\" from \"emps\" group by \"deptno\"",
         JdbcTest.HR_MODEL,
-        OptiqAssert.checkResultContains(
+        CalciteAssert.checkResultContains(
             "xxx"));
   }
 
@@ -288,8 +290,8 @@ public class MaterializationTest {
   @Test public void testMaterializationReferencesTableInOtherSchema() {}
 
   /** Unit test for logic functions
-   * {@link org.eigenbase.relopt.SubstitutionVisitor#mayBeSatisfiable} and
-   * {@link org.eigenbase.relopt.SubstitutionVisitor#simplify}. */
+   * {@link org.apache.calcite.plan.SubstitutionVisitor#mayBeSatisfiable} and
+   * {@link org.apache.calcite.plan.SubstitutionVisitor#simplify}. */
   @Test public void testSatisfiable() {
     // TRUE may be satisfiable
     checkSatisfiable(rexBuilder.makeLiteral(true), "true");
@@ -555,8 +557,8 @@ public class MaterializationTest {
    */
   @Ignore
   @Test public void testFilterGroupQueryOnStar() {
-    checkMaterialize(
-        "select p.\"product_name\", t.\"the_year\", sum(f.\"unit_sales\") as \"sum_unit_sales\", count(*) as \"c\"\n"
+    checkMaterialize("select p.\"product_name\", t.\"the_year\",\n"
+        + "  sum(f.\"unit_sales\") as \"sum_unit_sales\", count(*) as \"c\"\n"
         + "from \"foodmart\".\"sales_fact_1997\" as f\n"
         + "join (\n"
         + "    select \"time_id\", \"the_year\", \"the_month\"\n"
@@ -572,14 +574,14 @@ public class MaterializationTest {
         + " pc.\"product_category\",\n"
         + " p.\"product_name\"",
         "select t.\"the_month\", count(*) as x\n"
-        + "from (\n"
-        + "  select \"time_id\", \"the_year\", \"the_month\"\n"
-        + "  from \"foodmart\".\"time_by_day\") as t,\n"
-        + " \"foodmart\".\"sales_fact_1997\" as f\n"
-        + "where t.\"the_year\" = 1997\n"
-        + "and t.\"time_id\" = f.\"time_id\"\n"
-        + "group by t.\"the_year\",\n"
-        + " t.\"the_month\"\n",
+            + "from (\n"
+            + "  select \"time_id\", \"the_year\", \"the_month\"\n"
+            + "  from \"foodmart\".\"time_by_day\") as t,\n"
+            + " \"foodmart\".\"sales_fact_1997\" as f\n"
+            + "where t.\"the_year\" = 1997\n"
+            + "and t.\"time_id\" = f.\"time_id\"\n"
+            + "group by t.\"the_year\",\n"
+            + " t.\"the_month\"\n",
         JdbcTest.FOODMART_MODEL,
         CONTAINS_M0);
   }
@@ -588,8 +590,7 @@ public class MaterializationTest {
    * materialization that is just a join. */
   @Ignore
   @Test public void testQueryOnStar() {
-    String q =
-        "select *\n"
+    String q = "select *\n"
         + "from \"foodmart\".\"sales_fact_1997\" as f\n"
         + "join \"foodmart\".\"time_by_day\" as t on f.\"time_id\" = t.\"time_id\"\n"
         + "join \"foodmart\".\"product\" as p on f.\"product_id\" = p.\"product_id\"\n"
@@ -604,8 +605,7 @@ public class MaterializationTest {
    * nothing unpleasant happens. */
   @Ignore
   @Test public void testJoinOnUnionMaterialization() {
-    String q =
-        "select *\n"
+    String q = "select *\n"
         + "from (select * from \"emps\" union all select * from \"emps\")\n"
         + "join \"depts\" using (\"deptno\")";
     checkNoMaterialize(q, q, JdbcTest.HR_MODEL);

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/test/MockCatalogReader.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/MockCatalogReader.java b/core/src/test/java/org/apache/calcite/test/MockCatalogReader.java
index a51694f..7abe734 100644
--- a/core/src/test/java/org/apache/calcite/test/MockCatalogReader.java
+++ b/core/src/test/java/org/apache/calcite/test/MockCatalogReader.java
@@ -14,28 +14,48 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.test;
-
-import java.util.*;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.RelOptPlanner;
-import org.eigenbase.relopt.RelOptSchema;
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.sql.validate.*;
-import org.eigenbase.util.Pair;
-import org.eigenbase.util.Util;
-
-import net.hydromatic.linq4j.expressions.Expression;
-
-import net.hydromatic.optiq.prepare.Prepare;
+package org.apache.calcite.test;
+
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelOptSchema;
+import org.apache.calcite.prepare.Prepare;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollationImpl;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.logical.LogicalTableScan;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeComparability;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rel.type.RelDataTypeFieldImpl;
+import org.apache.calcite.sql.SqlAccessType;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.ObjectSqlType;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.validate.SqlMoniker;
+import org.apache.calcite.sql.validate.SqlMonikerImpl;
+import org.apache.calcite.sql.validate.SqlMonikerType;
+import org.apache.calcite.sql.validate.SqlMonotonicity;
+import org.apache.calcite.sql.validate.SqlValidatorCatalogReader;
+import org.apache.calcite.sql.validate.SqlValidatorUtil;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Ordering;
 
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.BitSet;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+
 /**
  * Mock implementation of {@link SqlValidatorCatalogReader} which returns tables
  * "EMP", "DEPT", "BONUS", "SALGRADE" (same as Oracle's SCOTT schema).
@@ -318,6 +338,7 @@ public class MockCatalogReader implements Prepare.CatalogReader {
 
   //~ Inner Classes ----------------------------------------------------------
 
+  /** Mock schema. */
   public static class MockSchema {
     private final List<String> tableNames = new ArrayList<String>();
     private String name;
@@ -337,7 +358,7 @@ public class MockCatalogReader implements Prepare.CatalogReader {
 
   /**
    * Mock implementation of
-   * {@link net.hydromatic.optiq.prepare.Prepare.PreparingTable}.
+   * {@link org.apache.calcite.prepare.Prepare.PreparingTable}.
    */
   public static class MockTable implements Prepare.PreparingTable {
     private final MockCatalogReader catalogReader;
@@ -373,7 +394,7 @@ public class MockCatalogReader implements Prepare.CatalogReader {
     }
 
     public RelNode toRel(ToRelContext context) {
-      return new TableAccessRel(context.getCluster(), this);
+      return new LogicalTableScan(context.getCluster(), this);
     }
 
     public List<RelCollation> getCollationList() {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/test/MockRelOptCost.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/MockRelOptCost.java b/core/src/test/java/org/apache/calcite/test/MockRelOptCost.java
index 5a4d4ca..4dbd98f 100644
--- a/core/src/test/java/org/apache/calcite/test/MockRelOptCost.java
+++ b/core/src/test/java/org/apache/calcite/test/MockRelOptCost.java
@@ -14,9 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.test;
+package org.apache.calcite.test;
 
-import org.eigenbase.relopt.*;
+import org.apache.calcite.plan.RelOptCost;
 
 /**
  * MockRelOptCost is a mock implementation of the {@link RelOptCost} interface.
@@ -25,15 +25,13 @@ import org.eigenbase.relopt.*;
 public class MockRelOptCost implements RelOptCost {
   //~ Methods ----------------------------------------------------------------
 
-  @Override
-  public boolean equals(Object obj) {
+  @Override public boolean equals(Object obj) {
     return this == obj
         || obj instanceof MockRelOptCost
         && equals((MockRelOptCost) obj);
   }
 
-  @Override
-  public int hashCode() {
+  @Override public int hashCode() {
     return 1;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/test/MockRelOptPlanner.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/MockRelOptPlanner.java b/core/src/test/java/org/apache/calcite/test/MockRelOptPlanner.java
index 102309b..ce61d21 100644
--- a/core/src/test/java/org/apache/calcite/test/MockRelOptPlanner.java
+++ b/core/src/test/java/org/apache/calcite/test/MockRelOptPlanner.java
@@ -14,16 +14,24 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.test;
-
-import java.util.*;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.rex.RexExecutorImpl;
-import org.eigenbase.util.Pair;
-
-import net.hydromatic.optiq.Schemas;
+package org.apache.calcite.test;
+
+import org.apache.calcite.plan.AbstractRelOptPlanner;
+import org.apache.calcite.plan.RelOptCostImpl;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptRuleOperand;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rex.RexExecutorImpl;
+import org.apache.calcite.schema.Schemas;
+import org.apache.calcite.util.Pair;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
 
 /**
  * MockRelOptPlanner is a mock implementation of the {@link RelOptPlanner}
@@ -185,8 +193,7 @@ public class MockRelOptPlanner extends AbstractRelOptPlanner {
     return true;
   }
 
-  @Override
-  public long getRelMetadataTimestamp(RelNode rel) {
+  @Override public long getRelMetadataTimestamp(RelNode rel) {
     return metadataTimestamp;
   }
 
@@ -197,6 +204,7 @@ public class MockRelOptPlanner extends AbstractRelOptPlanner {
 
   //~ Inner Classes ----------------------------------------------------------
 
+  /** Mock call to a planner rule. */
   private class MockRuleCall extends RelOptRuleCall {
     /**
      * Creates a MockRuleCall.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/test/MockSqlOperatorTable.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/MockSqlOperatorTable.java b/core/src/test/java/org/apache/calcite/test/MockSqlOperatorTable.java
index 5a077d2..02e7526 100644
--- a/core/src/test/java/org/apache/calcite/test/MockSqlOperatorTable.java
+++ b/core/src/test/java/org/apache/calcite/test/MockSqlOperatorTable.java
@@ -14,12 +14,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.test;
+package org.apache.calcite.test;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.sql.util.*;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.sql.SqlFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlOperatorBinding;
+import org.apache.calcite.sql.SqlOperatorTable;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.util.ChainedSqlOperatorTable;
+import org.apache.calcite.sql.util.ListSqlOperatorTable;
 
 import com.google.common.collect.ImmutableList;
 
@@ -55,10 +63,10 @@ public class MockSqlOperatorTable extends ChainedSqlOperatorTable {
     opTab.addOperator(new DedupFunction());
   }
 
+  /** "RAMP" user-defined function. */
   public static class RampFunction extends SqlFunction {
     public RampFunction() {
-      super(
-          "RAMP",
+      super("RAMP",
           SqlKind.OTHER_FUNCTION,
           null,
           null,
@@ -75,10 +83,10 @@ public class MockSqlOperatorTable extends ChainedSqlOperatorTable {
     }
   }
 
+  /** "DEDUP" user-defined function. */
   public static class DedupFunction extends SqlFunction {
     public DedupFunction() {
-      super(
-          "DEDUP",
+      super("DEDUP",
           SqlKind.OTHER_FUNCTION,
           null,
           null,

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/test/ModelTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/ModelTest.java b/core/src/test/java/org/apache/calcite/test/ModelTest.java
index d1ca838..13d12fc 100644
--- a/core/src/test/java/org/apache/calcite/test/ModelTest.java
+++ b/core/src/test/java/org/apache/calcite/test/ModelTest.java
@@ -14,9 +14,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.test;
+package org.apache.calcite.test;
 
-import net.hydromatic.optiq.model.*;
+import org.apache.calcite.model.JsonColumn;
+import org.apache.calcite.model.JsonCustomSchema;
+import org.apache.calcite.model.JsonJdbcSchema;
+import org.apache.calcite.model.JsonLattice;
+import org.apache.calcite.model.JsonMapSchema;
+import org.apache.calcite.model.JsonRoot;
+import org.apache.calcite.model.JsonTable;
+import org.apache.calcite.model.JsonView;
 
 import com.fasterxml.jackson.core.JsonParser;
 import com.fasterxml.jackson.databind.ObjectMapper;
@@ -24,10 +31,14 @@ import com.fasterxml.jackson.databind.ObjectMapper;
 import org.junit.Test;
 
 import java.io.IOException;
-import java.util.*;
+import java.util.List;
 
 import static org.hamcrest.CoreMatchers.equalTo;
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 /**
  * Unit test for data models.
@@ -113,18 +124,17 @@ public class ModelTest {
   /** Reads a custom schema. */
   @Test public void testCustomSchema() throws IOException {
     final ObjectMapper mapper = mapper();
-    JsonRoot root = mapper.readValue(
-        "{\n"
-        + "  version: '1.0',\n"
-        + "   schemas: [\n"
-        + "     {\n"
-        + "       type: 'custom',\n"
-        + "       name: 'My Custom Schema',\n"
-        + "       factory: 'com.acme.MySchemaFactory',\n"
-        + "       operand: {a: 'foo', b: [1, 3.5] }\n"
-        + "     }\n"
-        + "   ]\n"
-        + "}",
+    JsonRoot root = mapper.readValue("{\n"
+            + "  version: '1.0',\n"
+            + "   schemas: [\n"
+            + "     {\n"
+            + "       type: 'custom',\n"
+            + "       name: 'My Custom Schema',\n"
+            + "       factory: 'com.acme.MySchemaFactory',\n"
+            + "       operand: {a: 'foo', b: [1, 3.5] }\n"
+            + "     }\n"
+            + "   ]\n"
+            + "}",
         JsonRoot.class);
     assertEquals("1.0", root.version);
     assertEquals(1, root.schemas.size());
@@ -144,9 +154,8 @@ public class ModelTest {
    * materialization. */
   @Test public void testModelImmutableSchemaCannotContainMaterialization()
       throws Exception {
-    final OptiqAssert.AssertThat that =
-        OptiqAssert.that().withModel(
-            "{\n"
+    final CalciteAssert.AssertThat that =
+        CalciteAssert.that().withModel("{\n"
             + "  version: '1.0',\n"
             + "  defaultSchema: 'adhoc',\n"
             + "  schemas: [\n"
@@ -172,61 +181,59 @@ public class ModelTest {
             + "    }\n"
             + "  ]\n"
             + "}");
-    that.connectThrows(
-        "Cannot define materialization; parent schema 'adhoc' is not a "
-        + "SemiMutableSchema");
+    that.connectThrows("Cannot define materialization; parent schema 'adhoc' "
+        + "is not a SemiMutableSchema");
   }
 
   /** Tests a model containing a lattice and some views. */
   @Test public void testReadLattice() throws IOException {
     final ObjectMapper mapper = mapper();
-    JsonRoot root = mapper.readValue(
-        "{\n"
-        + "  version: '1.0',\n"
-        + "   schemas: [\n"
-        + "     {\n"
-        + "       name: 'FoodMart',\n"
-        + "       tables: [\n"
-        + "         {\n"
-        + "           name: 'time_by_day',\n"
-        + "           columns: [\n"
-        + "             {\n"
-        + "               name: 'time_id'\n"
-        + "             }\n"
-        + "           ]\n"
-        + "         },\n"
-        + "         {\n"
-        + "           name: 'sales_fact_1997',\n"
-        + "           columns: [\n"
-        + "             {\n"
-        + "               name: 'time_id'\n"
-        + "             }\n"
-        + "           ]\n"
-        + "         },\n"
-        + "         {\n"
-        + "           name: 'V',\n"
-        + "           type: 'view',\n"
-        + "           sql: 'values (1)'\n"
-        + "         },\n"
-        + "         {\n"
-        + "           name: 'V2',\n"
-        + "           type: 'view',\n"
-        + "           sql: [ 'values (1)', '(2)' ]\n"
-        + "         }\n"
-        + "       ],\n"
-        + "       lattices: [\n"
-        + "         {\n"
-        + "           name: 'SalesStar',\n"
-        + "           sql: 'select * from sales_fact_1997'\n"
-        + "         },\n"
-        + "         {\n"
-        + "           name: 'SalesStar2',\n"
-        + "           sql: [ 'select *', 'from sales_fact_1997' ]\n"
-        + "         }\n"
-        + "       ]\n"
-        + "     }\n"
-        + "   ]\n"
-        + "}",
+    JsonRoot root = mapper.readValue("{\n"
+            + "  version: '1.0',\n"
+            + "   schemas: [\n"
+            + "     {\n"
+            + "       name: 'FoodMart',\n"
+            + "       tables: [\n"
+            + "         {\n"
+            + "           name: 'time_by_day',\n"
+            + "           columns: [\n"
+            + "             {\n"
+            + "               name: 'time_id'\n"
+            + "             }\n"
+            + "           ]\n"
+            + "         },\n"
+            + "         {\n"
+            + "           name: 'sales_fact_1997',\n"
+            + "           columns: [\n"
+            + "             {\n"
+            + "               name: 'time_id'\n"
+            + "             }\n"
+            + "           ]\n"
+            + "         },\n"
+            + "         {\n"
+            + "           name: 'V',\n"
+            + "           type: 'view',\n"
+            + "           sql: 'values (1)'\n"
+            + "         },\n"
+            + "         {\n"
+            + "           name: 'V2',\n"
+            + "           type: 'view',\n"
+            + "           sql: [ 'values (1)', '(2)' ]\n"
+            + "         }\n"
+            + "       ],\n"
+            + "       lattices: [\n"
+            + "         {\n"
+            + "           name: 'SalesStar',\n"
+            + "           sql: 'select * from sales_fact_1997'\n"
+            + "         },\n"
+            + "         {\n"
+            + "           name: 'SalesStar2',\n"
+            + "           sql: [ 'select *', 'from sales_fact_1997' ]\n"
+            + "         }\n"
+            + "       ]\n"
+            + "     }\n"
+            + "   ]\n"
+            + "}",
         JsonRoot.class);
     assertEquals("1.0", root.version);
     assertEquals(1, root.schemas.size());
@@ -253,22 +260,21 @@ public class ModelTest {
   /** Tests a model with bad multi-line SQL. */
   @Test public void testReadBadMultiLineSql() throws IOException {
     final ObjectMapper mapper = mapper();
-    JsonRoot root = mapper.readValue(
-        "{\n"
-        + "  version: '1.0',\n"
-        + "   schemas: [\n"
-        + "     {\n"
-        + "       name: 'FoodMart',\n"
-        + "       tables: [\n"
-        + "         {\n"
-        + "           name: 'V',\n"
-        + "           type: 'view',\n"
-        + "           sql: [ 'values (1)', 2 ]\n"
-        + "         }\n"
-        + "       ]\n"
-        + "     }\n"
-        + "   ]\n"
-        + "}",
+    JsonRoot root = mapper.readValue("{\n"
+            + "  version: '1.0',\n"
+            + "   schemas: [\n"
+            + "     {\n"
+            + "       name: 'FoodMart',\n"
+            + "       tables: [\n"
+            + "         {\n"
+            + "           name: 'V',\n"
+            + "           type: 'view',\n"
+            + "           sql: [ 'values (1)', 2 ]\n"
+            + "         }\n"
+            + "       ]\n"
+            + "     }\n"
+            + "   ]\n"
+            + "}",
         JsonRoot.class);
     assertEquals(1, root.schemas.size());
     final JsonMapSchema schema = (JsonMapSchema) root.schemas.get(0);

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/test/MultiJdbcSchemaJoinTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/MultiJdbcSchemaJoinTest.java b/core/src/test/java/org/apache/calcite/test/MultiJdbcSchemaJoinTest.java
index c46ce79..426b531 100644
--- a/core/src/test/java/org/apache/calcite/test/MultiJdbcSchemaJoinTest.java
+++ b/core/src/test/java/org/apache/calcite/test/MultiJdbcSchemaJoinTest.java
@@ -14,25 +14,30 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.test;
+package org.apache.calcite.test;
 
-import net.hydromatic.optiq.SchemaPlus;
-import net.hydromatic.optiq.impl.java.ReflectiveSchema;
-import net.hydromatic.optiq.impl.jdbc.JdbcSchema;
-import net.hydromatic.optiq.jdbc.OptiqConnection;
-import net.hydromatic.optiq.prepare.OptiqPrepareImpl;
+import org.apache.calcite.adapter.java.ReflectiveSchema;
+import org.apache.calcite.adapter.jdbc.JdbcSchema;
+import org.apache.calcite.jdbc.CalciteConnection;
+import org.apache.calcite.prepare.CalcitePrepareImpl;
+import org.apache.calcite.schema.SchemaPlus;
 
 import com.google.common.collect.Sets;
 
 import org.junit.Test;
 
-import java.sql.*;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicInteger;
 import javax.sql.DataSource;
 
-import static org.hamcrest.CoreMatchers.*;
-import static org.junit.Assert.*;
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
 
 /** Test case for joining tables from two different JDBC databases. */
 public class MultiJdbcSchemaJoinTest {
@@ -42,25 +47,24 @@ public class MultiJdbcSchemaJoinTest {
     final String db1 = TempDb.INSTANCE.getUrl();
     Connection c1 = DriverManager.getConnection(db1, "", "");
     Statement stmt1 = c1.createStatement();
-    stmt1.execute(
-        "create table table1(id varchar(10) not null primary key, "
-            + "field1 varchar(10))");
+    stmt1.execute("create table table1(id varchar(10) not null primary key, "
+        + "field1 varchar(10))");
     stmt1.execute("insert into table1 values('a', 'aaaa')");
     c1.close();
 
     final String db2 = TempDb.INSTANCE.getUrl();
     Connection c2 = DriverManager.getConnection(db2, "", "");
     Statement stmt2 = c2.createStatement();
-    stmt2.execute(
-        "create table table2(id varchar(10) not null primary key, "
-            + "field1 varchar(10))");
+    stmt2.execute("create table table2(id varchar(10) not null primary key, "
+        + "field1 varchar(10))");
     stmt2.execute("insert into table2 values('a', 'aaaa')");
     c2.close();
 
     // Connect via calcite to these databases
     Connection connection = DriverManager.getConnection("jdbc:calcite:");
-    OptiqConnection optiqConnection = connection.unwrap(OptiqConnection.class);
-    SchemaPlus rootSchema = optiqConnection.getRootSchema();
+    CalciteConnection calciteConnection =
+        connection.unwrap(CalciteConnection.class);
+    SchemaPlus rootSchema = calciteConnection.getRootSchema();
     final DataSource ds1 =
         JdbcSchema.dataSource(db1, "org.hsqldb.jdbcDriver", "", "");
     rootSchema.add("DB1",
@@ -71,10 +75,9 @@ public class MultiJdbcSchemaJoinTest {
         JdbcSchema.create(rootSchema, "DB2", ds2, null, null));
 
     Statement stmt3 = connection.createStatement();
-    ResultSet rs = stmt3.executeQuery(
-        "select table1.id, table1.field1 "
-            + "from db1.table1 join db2.table2 on table1.id = table2.id");
-    assertThat(OptiqAssert.toString(rs), equalTo("ID=a; FIELD1=aaaa\n"));
+    ResultSet rs = stmt3.executeQuery("select table1.id, table1.field1 "
+        + "from db1.table1 join db2.table2 on table1.id = table2.id");
+    assertThat(CalciteAssert.toString(rs), equalTo("ID=a; FIELD1=aaaa\n"));
   }
 
   /** Makes sure that {@link #test} is re-entrant.
@@ -90,22 +93,22 @@ public class MultiJdbcSchemaJoinTest {
     Statement stmt1 = c1.createStatement();
     // This is a table we can join with the emps from the hr schema
     stmt1.execute("create table table1(id integer not null primary key, "
-            + "field1 varchar(10))");
+        + "field1 varchar(10))");
     stmt1.execute("insert into table1 values(100, 'foo')");
     stmt1.execute("insert into table1 values(200, 'bar')");
     c1.close();
 
     // Make a Calcite schema with both a jdbc schema and a non-jdbc schema
-    Connection optiqConn = DriverManager.getConnection("jdbc:calcite:");
-    OptiqConnection optiqConnection =
-        optiqConn.unwrap(OptiqConnection.class);
-    SchemaPlus rootSchema = optiqConnection.getRootSchema();
+    Connection connection = DriverManager.getConnection("jdbc:calcite:");
+    CalciteConnection calciteConnection =
+        connection.unwrap(CalciteConnection.class);
+    SchemaPlus rootSchema = calciteConnection.getRootSchema();
     rootSchema.add("DB",
         JdbcSchema.create(rootSchema, "DB",
             JdbcSchema.dataSource(db, "org.hsqldb.jdbcDriver", "", ""),
             null, null));
     rootSchema.add("hr", new ReflectiveSchema(new JdbcTest.HrSchema()));
-    return optiqConn;
+    return connection;
   }
 
   @Test public void testJdbcWithEnumerableJoin() throws SQLException {
@@ -141,13 +144,13 @@ public class MultiJdbcSchemaJoinTest {
     assertThat(runQuery(setup(), query), equalTo(expected));
   }
 
-  private Set<Integer> runQuery(Connection optiqConn, String query)
+  private Set<Integer> runQuery(Connection calciteConnection, String query)
       throws SQLException {
     // Print out the plan
-    Statement stmt = optiqConn.createStatement();
+    Statement stmt = calciteConnection.createStatement();
     try {
       ResultSet rs;
-      if (OptiqPrepareImpl.DEBUG) {
+      if (CalcitePrepareImpl.DEBUG) {
         rs = stmt.executeQuery("explain plan for " + query);
         rs.next();
         System.out.println(rs.getString(1));
@@ -170,14 +173,14 @@ public class MultiJdbcSchemaJoinTest {
     final String db = TempDb.INSTANCE.getUrl();
     Connection c1 = DriverManager.getConnection(db, "", "");
     Statement stmt1 = c1.createStatement();
-    stmt1.execute(
-        "create table table1(id varchar(10) not null primary key, "
-            + "field1 varchar(10))");
+    stmt1.execute("create table table1(id varchar(10) not null primary key, "
+        + "field1 varchar(10))");
 
     // Connect via calcite to these databases
     Connection connection = DriverManager.getConnection("jdbc:calcite:");
-    OptiqConnection optiqConnection = connection.unwrap(OptiqConnection.class);
-    SchemaPlus rootSchema = optiqConnection.getRootSchema();
+    CalciteConnection calciteConnection =
+        connection.unwrap(CalciteConnection.class);
+    SchemaPlus rootSchema = calciteConnection.getRootSchema();
     final DataSource ds =
         JdbcSchema.dataSource(db, "org.hsqldb.jdbcDriver", "", "");
     final SchemaPlus s =
@@ -196,9 +199,8 @@ public class MultiJdbcSchemaJoinTest {
           equalTo("Table 'DB.TABLE2' not found"));
     }
 
-    stmt1.execute(
-        "create table table2(id varchar(10) not null primary key, "
-            + "field1 varchar(10))");
+    stmt1.execute("create table table2(id varchar(10) not null primary key, "
+        + "field1 varchar(10))");
     stmt1.execute("insert into table2 values('a', 'aaaa')");
 
     // fails, table not visible due to caching
@@ -213,7 +215,7 @@ public class MultiJdbcSchemaJoinTest {
     // disable caching and table becomes visible
     s.setCacheEnabled(false);
     rs = stmt3.executeQuery("select * from db.table2");
-    assertThat(OptiqAssert.toString(rs), equalTo("ID=a; FIELD1=aaaa\n"));
+    assertThat(CalciteAssert.toString(rs), equalTo("ID=a; FIELD1=aaaa\n"));
     c1.close();
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/test/ReflectiveSchemaTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/ReflectiveSchemaTest.java b/core/src/test/java/org/apache/calcite/test/ReflectiveSchemaTest.java
index bd9a91f..0ff6ebc 100644
--- a/core/src/test/java/org/apache/calcite/test/ReflectiveSchemaTest.java
+++ b/core/src/test/java/org/apache/calcite/test/ReflectiveSchemaTest.java
@@ -14,32 +14,48 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.test;
-
-import net.hydromatic.linq4j.*;
-import net.hydromatic.linq4j.expressions.*;
-import net.hydromatic.linq4j.expressions.Types;
-import net.hydromatic.linq4j.function.*;
-
-import net.hydromatic.optiq.SchemaPlus;
-import net.hydromatic.optiq.impl.*;
-import net.hydromatic.optiq.impl.java.*;
-import net.hydromatic.optiq.jdbc.OptiqConnection;
-
-import org.eigenbase.util14.DateTimeUtil;
+package org.apache.calcite.test;
+
+import org.apache.calcite.adapter.java.ReflectiveSchema;
+import org.apache.calcite.jdbc.CalciteConnection;
+import org.apache.calcite.linq4j.Enumerable;
+import org.apache.calcite.linq4j.Linq4j;
+import org.apache.calcite.linq4j.QueryProvider;
+import org.apache.calcite.linq4j.function.Function1;
+import org.apache.calcite.linq4j.function.Predicate1;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.linq4j.tree.ParameterExpression;
+import org.apache.calcite.linq4j.tree.Primitive;
+import org.apache.calcite.linq4j.tree.Types;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.schema.impl.AbstractSchema;
+import org.apache.calcite.schema.impl.TableMacroImpl;
+import org.apache.calcite.schema.impl.ViewTable;
+import org.apache.calcite.util.DateTimeUtil;
 
 import org.junit.Ignore;
 import org.junit.Test;
 
-import java.lang.reflect.*;
-import java.sql.*;
+import java.lang.reflect.Field;
+import java.lang.reflect.Method;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
 import java.sql.Statement;
-import java.util.*;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.util.Arrays;
+import java.util.BitSet;
+import java.util.Collections;
 import java.util.Date;
+import java.util.List;
 
-import static net.hydromatic.optiq.test.JdbcTest.Employee;
+import static org.apache.calcite.test.JdbcTest.Employee;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 /**
  * Unit tests for {@link ReflectiveSchema}.
@@ -50,12 +66,13 @@ public class ReflectiveSchemaTest {
           Linq4j.class, "asEnumerable", Object[].class);
 
   /**
-   * Test that uses a JDBC connection as a linq4j {@link net.hydromatic.linq4j.QueryProvider}.
+   * Test that uses a JDBC connection as a linq4j
+   * {@link org.apache.calcite.linq4j.QueryProvider}.
    *
    * @throws Exception on error
    */
   @Test public void testQueryProvider() throws Exception {
-    Connection connection = OptiqAssert.getConnection("hr", "foodmart");
+    Connection connection = CalciteAssert.getConnection("hr", "foodmart");
     QueryProvider queryProvider = connection.unwrap(QueryProvider.class);
     ParameterExpression e = Expressions.parameter(Employee.class, "e");
 
@@ -104,7 +121,7 @@ public class ReflectiveSchemaTest {
   }
 
   @Test public void testQueryProviderSingleColumn() throws Exception {
-    Connection connection = OptiqAssert.getConnection("hr", "foodmart");
+    Connection connection = CalciteAssert.getConnection("hr", "foodmart");
     QueryProvider queryProvider = connection.unwrap(QueryProvider.class);
     ParameterExpression e = Expressions.parameter(Employee.class, "e");
 
@@ -129,16 +146,15 @@ public class ReflectiveSchemaTest {
 
   /**
    * Tests a relation that is accessed via method syntax.
-   * The function returns a {@link net.hydromatic.linq4j.Queryable}.
+   * The function returns a {@link org.apache.calcite.linq4j.Queryable}.
    */
   @Ignore
   @Test public void testOperator() throws SQLException, ClassNotFoundException {
-    Class.forName("net.hydromatic.optiq.jdbc.Driver");
     Connection connection =
         DriverManager.getConnection("jdbc:calcite:");
-    OptiqConnection optiqConnection =
-        connection.unwrap(OptiqConnection.class);
-    SchemaPlus rootSchema = optiqConnection.getRootSchema();
+    CalciteConnection calciteConnection =
+        connection.unwrap(CalciteConnection.class);
+    SchemaPlus rootSchema = calciteConnection.getRootSchema();
     SchemaPlus schema = rootSchema.add("s", new AbstractSchema());
     schema.add("GenerateStrings",
         TableMacroImpl.create(JdbcTest.GENERATE_STRINGS_METHOD));
@@ -158,12 +174,11 @@ public class ReflectiveSchemaTest {
    * Tests a view.
    */
   @Test public void testView() throws SQLException, ClassNotFoundException {
-    Class.forName("net.hydromatic.optiq.jdbc.Driver");
     Connection connection =
         DriverManager.getConnection("jdbc:calcite:");
-    OptiqConnection optiqConnection =
-        connection.unwrap(OptiqConnection.class);
-    SchemaPlus rootSchema = optiqConnection.getRootSchema();
+    CalciteConnection calciteConnection =
+        connection.unwrap(CalciteConnection.class);
+    SchemaPlus rootSchema = calciteConnection.getRootSchema();
     SchemaPlus schema = rootSchema.add("s", new AbstractSchema());
     schema.add("emps_view",
         ViewTable.viewMacro(schema,
@@ -177,19 +192,18 @@ public class ReflectiveSchemaTest {
     assertEquals(
         "empid=100; deptno=10; name=Bill; salary=10000.0; commission=1000\n"
         + "empid=110; deptno=10; name=Theodore; salary=11500.0; commission=250\n",
-        OptiqAssert.toString(resultSet));
+        CalciteAssert.toString(resultSet));
   }
 
   /**
    * Tests a view with a path.
    */
   @Test public void testViewPath() throws SQLException, ClassNotFoundException {
-    Class.forName("net.hydromatic.optiq.jdbc.Driver");
     Connection connection =
         DriverManager.getConnection("jdbc:calcite:");
-    OptiqConnection optiqConnection =
-        connection.unwrap(OptiqConnection.class);
-    SchemaPlus rootSchema = optiqConnection.getRootSchema();
+    CalciteConnection calciteConnection =
+        connection.unwrap(CalciteConnection.class);
+    SchemaPlus rootSchema = calciteConnection.getRootSchema();
     SchemaPlus schema = rootSchema.add("s", new AbstractSchema());
     // create a view s.emps based on hr.emps. uses explicit schema path "hr".
     schema.add("emps",
@@ -232,18 +246,18 @@ public class ReflectiveSchemaTest {
 
   /** Tests column based on java.sql.Date field. */
   @Test public void testDateColumn() throws Exception {
-    OptiqAssert.that()
+    CalciteAssert.that()
         .with("s", new DateColumnSchema())
         .query("select * from \"s\".\"emps\"")
-        .returns(
-            "hireDate=1970-01-01; empid=10; deptno=20; name=fred; salary=0.0; commission=null\n"
+        .returns(""
+            + "hireDate=1970-01-01; empid=10; deptno=20; name=fred; salary=0.0; commission=null\n"
             + "hireDate=1970-04-11; empid=10; deptno=20; name=bill; salary=0.0; commission=null\n");
   }
 
   /** Tests querying an object that has no public fields. */
   @Test public void testNoPublicFields() throws Exception {
-    final OptiqAssert.AssertThat with =
-        OptiqAssert.that().with("s", new CatchallSchema());
+    final CalciteAssert.AssertThat with =
+        CalciteAssert.that().with("s", new CatchallSchema());
     with.query("select 1 from \"s\".\"allPrivates\"")
         .returns("EXPR$0=1\n");
     with.query("select \"x\" from \"s\".\"allPrivates\"")
@@ -254,15 +268,14 @@ public class ReflectiveSchemaTest {
    *
    * @see CatchallSchema#everyTypes */
   @Test public void testColumnTypes() throws Exception {
-    final OptiqAssert.AssertThat with =
-        OptiqAssert.that().with("s", new CatchallSchema());
+    final CalciteAssert.AssertThat with =
+        CalciteAssert.that().with("s", new CatchallSchema());
     with.query("select \"primitiveBoolean\" from \"s\".\"everyTypes\"")
-        .returns(
-            "primitiveBoolean=false\n"
+        .returns("primitiveBoolean=false\n"
             + "primitiveBoolean=true\n");
     with.query("select * from \"s\".\"everyTypes\"")
-        .returns(
-            "primitiveBoolean=false; primitiveByte=0; primitiveChar=\u0000; primitiveShort=0; primitiveInt=0; primitiveLong=0; primitiveFloat=0.0; primitiveDouble=0.0; wrapperBoolean=false; wrapperByte=0; wrapperCharacter=\u0000; wrapperShort=0; wrapperInteger=0; wrapperLong=0; wrapperFloat=0.0; wrapperDouble=0.0; sqlDate=1970-01-01; sqlTime=00:00:00; sqlTimestamp=1970-01-01 00:00:00; utilDate=1970-01-01 00:00:00; string=1\n"
+        .returns(""
+            + "primitiveBoolean=false; primitiveByte=0; primitiveChar=\u0000; primitiveShort=0; primitiveInt=0; primitiveLong=0; primitiveFloat=0.0; primitiveDouble=0.0; wrapperBoolean=false; wrapperByte=0; wrapperCharacter=\u0000; wrapperShort=0; wrapperInteger=0; wrapperLong=0; wrapperFloat=0.0; wrapperDouble=0.0; sqlDate=1970-01-01; sqlTime=00:00:00; sqlTimestamp=1970-01-01 00:00:00; utilDate=1970-01-01 00:00:00; string=1\n"
             + "primitiveBoolean=true; primitiveByte=127; primitiveChar=\uffff; primitiveShort=32767; primitiveInt=2147483647; primitiveLong=9223372036854775807; primitiveFloat=3.4028235E38; primitiveDouble=1.7976931348623157E308; wrapperBoolean=null; wrapperByte=null; wrapperCharacter=null; wrapperShort=null; wrapperInteger=null; wrapperLong=null; wrapperFloat=null; wrapperDouble=null; sqlDate=null; sqlTime=null; sqlTimestamp=null; utilDate=null; string=null\n");
   }
 
@@ -270,8 +283,8 @@ public class ReflectiveSchemaTest {
    *
    * @see CatchallSchema#everyTypes */
   @Test public void testAggregateFunctions() throws Exception {
-    final OptiqAssert.AssertThat with =
-        OptiqAssert.that()
+    final CalciteAssert.AssertThat with =
+        CalciteAssert.that()
             .with("s", new CatchallSchema());
     checkAgg(with, "min");
     checkAgg(with, "max");
@@ -279,33 +292,28 @@ public class ReflectiveSchemaTest {
     checkAgg(with, "count");
   }
 
-  private void checkAgg(OptiqAssert.AssertThat with, String fn) {
+  private void checkAgg(CalciteAssert.AssertThat with, String fn) {
     for (Field field
         : fn.equals("avg") ? EveryType.numericFields() : EveryType.fields()) {
-      with.query(
-          "select " + fn + "(\"" + field.getName() + "\") as c\n"
+      with.query("select " + fn + "(\"" + field.getName() + "\") as c\n"
           + "from \"s\".\"everyTypes\"")
-          .returns(OptiqAssert.<ResultSet, Void>constantNull());
+          .returns(CalciteAssert.<ResultSet, Void>constantNull());
     }
   }
 
   @Test public void testJavaBoolean() throws Exception {
-    final OptiqAssert.AssertThat with =
-        OptiqAssert.that().with("s", new CatchallSchema());
-    with.query(
-        "select count(*) as c from \"s\".\"everyTypes\"\n"
+    final CalciteAssert.AssertThat with =
+        CalciteAssert.that().with("s", new CatchallSchema());
+    with.query("select count(*) as c from \"s\".\"everyTypes\"\n"
         + "where \"primitiveBoolean\"")
         .returns("C=1\n");
-    with.query(
-        "select count(*) as c from \"s\".\"everyTypes\"\n"
+    with.query("select count(*) as c from \"s\".\"everyTypes\"\n"
         + "where \"wrapperBoolean\"")
         .returns("C=0\n");
-    with.query(
-        "select count(*) as c from \"s\".\"everyTypes\"\n"
+    with.query("select count(*) as c from \"s\".\"everyTypes\"\n"
         + "where \"wrapperBoolean\" is not true")
         .returns("C=2\n");
-    with.query(
-        "select count(*) as c from \"s\".\"everyTypes\"\n"
+    with.query("select count(*) as c from \"s\".\"everyTypes\"\n"
         + "where \"primitiveInt\" > 0")
         .returns("C=1\n");
   }
@@ -314,19 +322,17 @@ public class ReflectiveSchemaTest {
    * <a href="https://issues.apache.org/jira/browse/CALCITE-119">CALCITE-119</a>,
    * "Comparing a Java type long with a SQL type INTEGER gives wrong answer". */
   @Test public void testCompareJavaAndSqlTypes() throws Exception {
-    final OptiqAssert.AssertThat with =
-        OptiqAssert.that().with("s", new CatchallSchema());
+    final CalciteAssert.AssertThat with =
+        CalciteAssert.that().with("s", new CatchallSchema());
     // With CALCITE-119, returned 0 rows. The problem was that when comparing
     // a Java type (long) and a SQL type (INTEGER), the SQL type was deemed
     // "less restrictive". So, the long value got truncated to an int value.
-    with.query(
-        "select \"primitiveLong\" as c from \"s\".\"everyTypes\"\n"
+    with.query("select \"primitiveLong\" as c from \"s\".\"everyTypes\"\n"
         + "where \"primitiveLong\" > 0")
         .returns("C=9223372036854775807\n");
 
     // count(nullif(b, false)) counts how many times b is true
-    with.query(
-        "select count(\"primitiveBoolean\") as p,\n"
+    with.query("select count(\"primitiveBoolean\") as p,\n"
         + "  count(\"wrapperBoolean\") as w,\n"
         + "  count(nullif(\"primitiveShort\" >= 0, false)) as sp,\n"
         + "  count(nullif(\"wrapperShort\" >= 0, false)) as sw,\n"
@@ -339,10 +345,9 @@ public class ReflectiveSchemaTest {
   }
 
   @Test public void testDivideWraperPrimitive() throws Exception {
-    final OptiqAssert.AssertThat with =
-        OptiqAssert.that().with("s", new CatchallSchema());
-    with.query(
-        "select \"wrapperLong\" / \"primitiveLong\" as c\n"
+    final CalciteAssert.AssertThat with =
+        CalciteAssert.that().with("s", new CatchallSchema());
+    with.query("select \"wrapperLong\" / \"primitiveLong\" as c\n"
         + " from \"s\".\"everyTypes\" where \"primitiveLong\" <> 0")
         .planContains(
             "final Long inp13_ = current.wrapperLong;")
@@ -352,35 +357,33 @@ public class ReflectiveSchemaTest {
   }
 
   @Test public void testDivideWraperWrapper() throws Exception {
-    final OptiqAssert.AssertThat with =
-        OptiqAssert.that().with("s", new CatchallSchema());
-    with.query(
-        "select \"wrapperLong\" / \"wrapperLong\" as c\n"
+    final CalciteAssert.AssertThat with =
+        CalciteAssert.that().with("s", new CatchallSchema());
+    with.query("select \"wrapperLong\" / \"wrapperLong\" as c\n"
         + " from \"s\".\"everyTypes\" where \"primitiveLong\" <> 0")
         .planContains(
-            "final Long inp13_ = ((net.hydromatic.optiq.test.ReflectiveSchemaTest.EveryType) inputEnumerator.current()).wrapperLong;")
+            "final Long inp13_ = ((org.apache.calcite.test.ReflectiveSchemaTest.EveryType) inputEnumerator.current()).wrapperLong;")
         .planContains(
             "return inp13_ == null ? (Long) null : Long.valueOf(inp13_.longValue() / inp13_.longValue());")
         .returns("C=null\n");
   }
 
   @Test public void testDivideWraperWrapperMultipleTimes() throws Exception {
-    final OptiqAssert.AssertThat with =
-        OptiqAssert.that().with("s", new CatchallSchema());
-    with.query(
-        "select \"wrapperLong\" / \"wrapperLong\"\n"
+    final CalciteAssert.AssertThat with =
+        CalciteAssert.that().with("s", new CatchallSchema());
+    with.query("select \"wrapperLong\" / \"wrapperLong\"\n"
         + "+ \"wrapperLong\" / \"wrapperLong\" as c\n"
         + " from \"s\".\"everyTypes\" where \"primitiveLong\" <> 0")
         .planContains(
-            "final Long inp13_ = ((net.hydromatic.optiq.test.ReflectiveSchemaTest.EveryType) inputEnumerator.current()).wrapperLong;")
+            "final Long inp13_ = ((org.apache.calcite.test.ReflectiveSchemaTest.EveryType) inputEnumerator.current()).wrapperLong;")
         .planContains(
             "return inp13_ == null ? (Long) null : Long.valueOf(inp13_.longValue() / inp13_.longValue() + inp13_.longValue() / inp13_.longValue());")
         .returns("C=null\n");
   }
 
   @Test public void testOp() throws Exception {
-    final OptiqAssert.AssertThat with =
-        OptiqAssert.that()
+    final CalciteAssert.AssertThat with =
+        CalciteAssert.that()
             .with("s", new CatchallSchema());
     checkOp(with, "+");
     checkOp(with, "-");
@@ -388,28 +391,25 @@ public class ReflectiveSchemaTest {
     checkOp(with, "/");
   }
 
-  private void checkOp(OptiqAssert.AssertThat with, String fn) {
+  private void checkOp(CalciteAssert.AssertThat with, String fn) {
     for (Field field : EveryType.numericFields()) {
       for (Field field2 : EveryType.numericFields()) {
         final String name = "\"" + field.getName() + "\"";
         final String name2 = "\"" + field2.getName() + "\"";
-        with.query(
-            "select " + name + "\n"
+        with.query("select " + name + "\n"
             + " " + fn + " " + name2 + " as c\n"
             + "from \"s\".\"everyTypes\"\n"
             + "where " + name + " <> 0")
-            .returns(OptiqAssert.<ResultSet, Void>constantNull());
+            .returns(CalciteAssert.<ResultSet, Void>constantNull());
       }
     }
   }
 
   @Test public void testCastFromString() {
-    OptiqAssert.that()
+    CalciteAssert.that()
         .with("s", new CatchallSchema())
-        .query(
-            "select cast(\"string\" as int) as c from \"s\".\"everyTypes\"")
-        .returns(
-            "C=1\n"
+        .query("select cast(\"string\" as int) as c from \"s\".\"everyTypes\"")
+        .returns("C=1\n"
             + "C=null\n");
   }
 
@@ -429,7 +429,7 @@ public class ReflectiveSchemaTest {
    *
    * @see CatchallSchema#badTypes */
   @Test public void testTableFieldHasBadType() throws Exception {
-    OptiqAssert.that()
+    CalciteAssert.that()
         .with("s", new CatchallSchema())
         .query("select * from \"s\".\"badTypes\"")
         .returns("integer=0; bitSet={}\n");
@@ -441,8 +441,8 @@ public class ReflectiveSchemaTest {
    * @see CatchallSchema#enumerable
    * @see CatchallSchema#list */
   @Test public void testSchemaFieldHasBadType() throws Exception {
-    final OptiqAssert.AssertThat with =
-        OptiqAssert.that()
+    final CalciteAssert.AssertThat with =
+        CalciteAssert.that()
             .with("s", new CatchallSchema());
     // BitSet is not a valid relation type. It's as if "bitSet" field does
     // not exist.
@@ -450,15 +450,13 @@ public class ReflectiveSchemaTest {
         .throws_("Table 's.bitSet' not found");
     // Enumerable field returns 3 records with 0 fields
     with.query("select * from \"s\".\"enumerable\"")
-        .returns(
-            "\n"
+        .returns("\n"
             + "\n"
             + "\n"
             + "\n");
     // List is implicitly converted to Enumerable
     with.query("select * from \"s\".\"list\"")
-        .returns(
-            "\n"
+        .returns("\n"
             + "\n"
             + "\n"
             + "\n");
@@ -467,12 +465,11 @@ public class ReflectiveSchemaTest {
   /** Test case for a bug where a Java string 'Abc' compared to a char 'Ab'
    * would be truncated to the char precision and falsely match. */
   @Test public void testPrefix() throws Exception {
-    OptiqAssert.that()
+    CalciteAssert.that()
         .with("s", new CatchallSchema())
         .query(
             "select * from \"s\".\"prefixEmps\" where \"name\" in ('Ab', 'Abd')")
-        .returns(
-            "empid=2; deptno=10; name=Ab; salary=0.0; commission=null\n"
+        .returns("empid=2; deptno=10; name=Ab; salary=0.0; commission=null\n"
             + "empid=4; deptno=10; name=Abd; salary=0.0; commission=null\n");
   }
 
@@ -481,27 +478,24 @@ public class ReflectiveSchemaTest {
    * should be expanded. */
   @Ignore
   @Test public void testTableMacroIsView() throws Exception {
-    OptiqAssert.that()
+    CalciteAssert.that()
         .with("s", new JdbcTest.HrSchema())
-        .query(
-            "select * from table(\"s\".\"view\"('abc'))")
-        .returns(
-            "empid=2; deptno=10; name=Ab; salary=0.0; commission=null\n"
+        .query("select * from table(\"s\".\"view\"('abc'))")
+        .returns("empid=2; deptno=10; name=Ab; salary=0.0; commission=null\n"
             + "empid=4; deptno=10; name=Abd; salary=0.0; commission=null\n");
   }
 
   /** Finds a table-macro using reflection. */
   @Ignore
   @Test public void testTableMacro() throws Exception {
-    OptiqAssert.that()
+    CalciteAssert.that()
         .with("s", new JdbcTest.HrSchema())
-        .query(
-            "select * from table(\"s\".\"foo\"(3))")
-        .returns(
-            "empid=2; deptno=10; name=Ab; salary=0.0; commission=null\n"
+        .query("select * from table(\"s\".\"foo\"(3))")
+        .returns("empid=2; deptno=10; name=Ab; salary=0.0; commission=null\n"
             + "empid=4; deptno=10; name=Abd; salary=0.0; commission=null\n");
   }
 
+  /** Extension to {@link Employee} with a {@code hireDate} column. */
   public static class EmployeeWithHireDate extends Employee {
     public final java.sql.Date hireDate;
 
@@ -513,6 +507,7 @@ public class ReflectiveSchemaTest {
     }
   }
 
+  /** Record that has a field of every interesting type. */
   public static class EveryType {
     public final boolean primitiveBoolean;
     public final byte primitiveByte;
@@ -645,6 +640,7 @@ public class ReflectiveSchemaTest {
     };
   }
 
+  /** Schema that contains a table with a date column. */
   public static class DateColumnSchema {
     public final EmployeeWithHireDate[] emps = {
       new EmployeeWithHireDate(

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/test/RelMetadataTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/RelMetadataTest.java b/core/src/test/java/org/apache/calcite/test/RelMetadataTest.java
index 32e6dda..41caea0 100644
--- a/core/src/test/java/org/apache/calcite/test/RelMetadataTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RelMetadataTest.java
@@ -14,14 +14,22 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.test;
-
-import java.lang.reflect.Method;
-import java.util.*;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.rel.metadata.*;
-import org.eigenbase.relopt.*;
+package org.apache.calcite.test;
+
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Aggregate;
+import org.apache.calcite.rel.logical.LogicalAggregate;
+import org.apache.calcite.rel.logical.LogicalFilter;
+import org.apache.calcite.rel.metadata.CachingRelMetadataProvider;
+import org.apache.calcite.rel.metadata.ChainedRelMetadataProvider;
+import org.apache.calcite.rel.metadata.DefaultRelMetadataProvider;
+import org.apache.calcite.rel.metadata.Metadata;
+import org.apache.calcite.rel.metadata.ReflectiveRelMetadataProvider;
+import org.apache.calcite.rel.metadata.RelColumnOrigin;
+import org.apache.calcite.rel.metadata.RelMetadataProvider;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterables;
@@ -31,15 +39,24 @@ import org.hamcrest.Matcher;
 import org.junit.Ignore;
 import org.junit.Test;
 
-import static org.hamcrest.CoreMatchers.*;
-import static org.junit.Assert.*;
+import java.lang.reflect.Method;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.List;
+import java.util.Set;
+
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.CoreMatchers.instanceOf;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
 
 /**
- * Unit test for {@link DefaultRelMetadataProvider}. See {@link
- * SqlToRelTestBase} class comments for details on the schema used. Note that no
- * optimizer rules are fired on the translation of the SQL into relational
- * algebra (e.g. join conditions in the WHERE clause will look like filters), so
- * it's necessary to phrase the SQL carefully.
+ * Unit test for {@link DefaultRelMetadataProvider}. See
+ * {@link SqlToRelTestBase} class comments for details on the schema used. Note
+ * that no optimizer rules are fired on the translation of the SQL into
+ * relational algebra (e.g. join conditions in the WHERE clause will look like
+ * filters), so it's necessary to phrase the SQL carefully.
  */
 public class RelMetadataTest extends SqlToRelTestBase {
   //~ Static fields/initializers ---------------------------------------------
@@ -113,17 +130,17 @@ public class RelMetadataTest extends SqlToRelTestBase {
 
   @Ignore
   @Test public void testPercentageOriginalRowsTwoFilters() {
-    checkPercentageOriginalRows(
-        "select * from (select * from dept where name='X')"
-        + " where deptno = 20",
+    checkPercentageOriginalRows("select * from (\n"
+        + "  select * from dept where name='X')\n"
+        + "where deptno = 20",
         DEFAULT_EQUAL_SELECTIVITY_SQUARED);
   }
 
   @Ignore
   @Test public void testPercentageOriginalRowsRedundantFilter() {
-    checkPercentageOriginalRows(
-        "select * from (select * from dept where deptno=20)"
-        + " where deptno = 20",
+    checkPercentageOriginalRows("select * from (\n"
+        + "  select * from dept where deptno=20)\n"
+        + "where deptno = 20",
         DEFAULT_EQUAL_SELECTIVITY);
   }
 
@@ -135,10 +152,10 @@ public class RelMetadataTest extends SqlToRelTestBase {
 
   @Ignore
   @Test public void testPercentageOriginalRowsJoinTwoFilters() {
-    checkPercentageOriginalRows(
-        "select * from (select * from emp where deptno=10) e"
-        + " inner join (select * from dept where deptno=10) d"
-        + " on e.deptno=d.deptno",
+    checkPercentageOriginalRows("select * from (\n"
+        + "  select * from emp where deptno=10) e\n"
+        + "inner join (select * from dept where deptno=10) d\n"
+        + "on e.deptno=d.deptno",
         DEFAULT_EQUAL_SELECTIVITY_SQUARED);
   }
 
@@ -152,7 +169,7 @@ public class RelMetadataTest extends SqlToRelTestBase {
   @Test public void testPercentageOriginalRowsUnionLittleFilter() {
     checkPercentageOriginalRows(
         "select name from dept where deptno=20"
-        + " union all select ename from emp",
+            + " union all select ename from emp",
         ((DEPT_SIZE * DEFAULT_EQUAL_SELECTIVITY) + EMP_SIZE)
             / (DEPT_SIZE + EMP_SIZE));
   }
@@ -161,7 +178,7 @@ public class RelMetadataTest extends SqlToRelTestBase {
   @Test public void testPercentageOriginalRowsUnionBigFilter() {
     checkPercentageOriginalRows(
         "select name from dept"
-        + " union all select ename from emp where deptno=20",
+            + " union all select ename from emp where deptno=20",
         ((EMP_SIZE * DEFAULT_EQUAL_SELECTIVITY) + DEPT_SIZE)
             / (DEPT_SIZE + EMP_SIZE));
   }
@@ -306,7 +323,7 @@ public class RelMetadataTest extends SqlToRelTestBase {
   @Test public void testColumnOriginsJoinOuter() {
     checkSingleColumnOrigin(
         "select name as dname from emp left outer join dept"
-        + " on emp.deptno = dept.deptno",
+            + " on emp.deptno = dept.deptno",
         "DEPT",
         "NAME",
         true);
@@ -315,7 +332,7 @@ public class RelMetadataTest extends SqlToRelTestBase {
   @Test public void testColumnOriginsJoinFullOuter() {
     checkSingleColumnOrigin(
         "select name as dname from emp full outer join dept"
-        + " on emp.deptno = dept.deptno",
+            + " on emp.deptno = dept.deptno",
         "DEPT",
         "NAME",
         true);
@@ -482,24 +499,22 @@ public class RelMetadataTest extends SqlToRelTestBase {
 
   @Test public void testSelectivitySort() {
     RelNode rel =
-        convertSql(
-            "select * from emp where deptno = 10"
+        convertSql("select * from emp where deptno = 10"
             + "order by ename");
     checkRelSelectivity(rel, DEFAULT_EQUAL_SELECTIVITY);
   }
 
   @Test public void testSelectivityUnion() {
     RelNode rel =
-        convertSql(
-            "select * from (select * from emp union all select * from emp) "
-                + "where deptno = 10");
+        convertSql("select * from (\n"
+            + "  select * from emp union all select * from emp) "
+            + "where deptno = 10");
     checkRelSelectivity(rel, DEFAULT_EQUAL_SELECTIVITY);
   }
 
   @Test public void testSelectivityAgg() {
     RelNode rel =
-        convertSql(
-            "select deptno, count(*) from emp where deptno > 10 "
+        convertSql("select deptno, count(*) from emp where deptno > 10 "
             + "group by deptno having count(*) = 0");
     checkRelSelectivity(
         rel,
@@ -510,8 +525,7 @@ public class RelMetadataTest extends SqlToRelTestBase {
    * argument. */
   @Test public void testSelectivityAggCached() {
     RelNode rel =
-        convertSql(
-            "select deptno, count(*) from emp where deptno > 10 "
+        convertSql("select deptno, count(*) from emp where deptno > 10 "
             + "group by deptno having count(*) = 0");
     rel.getCluster().setMetadataProvider(
         new CachingRelMetadataProvider(
@@ -537,25 +551,24 @@ public class RelMetadataTest extends SqlToRelTestBase {
     ColTypeImpl.THREAD_LIST.set(buf);
 
     RelNode rel =
-        convertSql(
-            "select deptno, count(*) from emp where deptno > 10 "
-                + "group by deptno having count(*) = 0");
+        convertSql("select deptno, count(*) from emp where deptno > 10 "
+            + "group by deptno having count(*) = 0");
     rel.getCluster().setMetadataProvider(
         ChainedRelMetadataProvider.of(
             ImmutableList.of(
                 ColTypeImpl.SOURCE, rel.getCluster().getMetadataProvider())));
 
     // Top node is a filter. Its metadata uses getColType(RelNode, int).
-    assertThat(rel, instanceOf(FilterRel.class));
+    assertThat(rel, instanceOf(LogicalFilter.class));
     assertThat(rel.metadata(ColType.class).getColType(0),
         equalTo("DEPTNO-rel"));
     assertThat(rel.metadata(ColType.class).getColType(1),
         equalTo("EXPR$1-rel"));
 
     // Next node is an aggregate. Its metadata uses
-    // getColType(AggregateRel, int).
+    // getColType(LogicalAggregate, int).
     final RelNode input = rel.getInput(0);
-    assertThat(input, instanceOf(AggregateRel.class));
+    assertThat(input, instanceOf(LogicalAggregate.class));
     assertThat(input.metadata(ColType.class).getColType(0),
         equalTo("DEPTNO-agg"));
 
@@ -605,7 +618,7 @@ public class RelMetadataTest extends SqlToRelTestBase {
     String getColType(int column);
   }
 
-  /** A provider for {@link org.eigenbase.test.RelMetadataTest.ColType} via
+  /** A provider for {@link org.apache.calcite.test.RelMetadataTest.ColType} via
    * reflection. */
   public static class ColTypeImpl {
     static final ThreadLocal<List<String>> THREAD_LIST =
@@ -624,9 +637,10 @@ public class RelMetadataTest extends SqlToRelTestBase {
             METHOD, new ColTypeImpl());
 
     /** Implementation of {@link ColType#getColType(int)} for
-     * {@link AggregateRel}, called via reflection. */
+     * {@link org.apache.calcite.rel.logical.LogicalAggregate}, called via
+     * reflection. */
     @SuppressWarnings("UnusedDeclaration")
-    public String getColType(AggregateRelBase rel, int column) {
+    public String getColType(Aggregate rel, int column) {
       final String name =
           rel.getRowType().getFieldList().get(column).getName() + "-agg";
       THREAD_LIST.get().add(name);


[02/58] [abbrv] [partial] incubator-calcite git commit: [CALCITE-306] Standardize code style for "import package.*; "

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/test/java/org/apache/calcite/linq4j/test/ExpressionTest.java
----------------------------------------------------------------------
diff --git a/linq4j/src/test/java/org/apache/calcite/linq4j/test/ExpressionTest.java b/linq4j/src/test/java/org/apache/calcite/linq4j/test/ExpressionTest.java
index d868724..0be06c1 100644
--- a/linq4j/src/test/java/org/apache/calcite/linq4j/test/ExpressionTest.java
+++ b/linq4j/src/test/java/org/apache/calcite/linq4j/test/ExpressionTest.java
@@ -14,10 +14,25 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.test;
-
-import net.hydromatic.linq4j.expressions.*;
-import net.hydromatic.linq4j.function.Function1;
+package org.apache.calcite.linq4j.test;
+
+import org.apache.calcite.linq4j.function.Function1;
+import org.apache.calcite.linq4j.tree.BlockBuilder;
+import org.apache.calcite.linq4j.tree.BlockStatement;
+import org.apache.calcite.linq4j.tree.Blocks;
+import org.apache.calcite.linq4j.tree.ClassDeclaration;
+import org.apache.calcite.linq4j.tree.DeclarationStatement;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.linq4j.tree.FieldDeclaration;
+import org.apache.calcite.linq4j.tree.FunctionExpression;
+import org.apache.calcite.linq4j.tree.MemberDeclaration;
+import org.apache.calcite.linq4j.tree.MethodCallExpression;
+import org.apache.calcite.linq4j.tree.NewExpression;
+import org.apache.calcite.linq4j.tree.Node;
+import org.apache.calcite.linq4j.tree.ParameterExpression;
+import org.apache.calcite.linq4j.tree.Types;
+import org.apache.calcite.linq4j.tree.Visitor;
 
 import org.junit.Test;
 
@@ -25,12 +40,17 @@ import java.lang.reflect.Modifier;
 import java.lang.reflect.Type;
 import java.math.BigDecimal;
 import java.math.BigInteger;
-import java.util.*;
+import java.util.AbstractList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.TreeSet;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
 
 /**
- * Unit test for {@link net.hydromatic.linq4j.expressions.Expression}
+ * Unit test for {@link org.apache.calcite.linq4j.tree.Expression}
  * and subclasses.
  */
 public class ExpressionTest {
@@ -50,19 +70,19 @@ public class ExpressionTest {
     // Print out the expression.
     String s = Expressions.toString(lambdaExpr);
     assertEquals(
-        "new net.hydromatic.linq4j.function.Function1() {\n"
-        + "  public double apply(double arg) {\n"
-        + "    return arg + 2.0D;\n"
-        + "  }\n"
-        + "  public Object apply(Double arg) {\n"
-        + "    return apply(\n"
-        + "      arg.doubleValue());\n"
-        + "  }\n"
-        + "  public Object apply(Object arg) {\n"
-        + "    return apply(\n"
-        + "      (Double) arg);\n"
-        + "  }\n"
-        + "}\n",
+        "new org.apache.calcite.linq4j.function.Function1() {\n"
+            + "  public double apply(double arg) {\n"
+            + "    return arg + 2.0D;\n"
+            + "  }\n"
+            + "  public Object apply(Double arg) {\n"
+            + "    return apply(\n"
+            + "      arg.doubleValue());\n"
+            + "  }\n"
+            + "  public Object apply(Object arg) {\n"
+            + "    return apply(\n"
+            + "      (Double) arg);\n"
+            + "  }\n"
+            + "}\n",
         s);
 
     // Compile and run the lambda expression.
@@ -92,21 +112,21 @@ public class ExpressionTest {
 
     // Print out the expression.
     String s = Expressions.toString(lambdaExpr);
-    assertEquals("new net.hydromatic.linq4j.function.Function2() {\n"
-        + "  public int apply(int key, int key2) {\n"
-        + "    return key;\n"
-        + "  }\n"
-        + "  public Integer apply(Integer key, Integer key2) {\n"
-        + "    return apply(\n"
-        + "      key.intValue(),\n"
-        + "      key2.intValue());\n"
-        + "  }\n"
-        + "  public Integer apply(Object key, Object key2) {\n"
-        + "    return apply(\n"
-        + "      (Integer) key,\n"
-        + "      (Integer) key2);\n"
-        + "  }\n"
-        + "}\n",
+    assertEquals("new org.apache.calcite.linq4j.function.Function2() {\n"
+            + "  public int apply(int key, int key2) {\n"
+            + "    return key;\n"
+            + "  }\n"
+            + "  public Integer apply(Integer key, Integer key2) {\n"
+            + "    return apply(\n"
+            + "      key.intValue(),\n"
+            + "      key2.intValue());\n"
+            + "  }\n"
+            + "  public Integer apply(Object key, Object key2) {\n"
+            + "    return apply(\n"
+            + "      (Integer) key,\n"
+            + "      (Integer) key2);\n"
+            + "  }\n"
+            + "}\n",
         s);
   }
 
@@ -315,15 +335,15 @@ public class ExpressionTest {
     final ParameterExpression paramX =
         Expressions.parameter(String.class, "x");
     assertEquals(
-        "new net.hydromatic.linq4j.function.Function1() {\n"
-        + "  public int apply(String x) {\n"
-        + "    return x.length();\n"
-        + "  }\n"
-        + "  public Object apply(Object x) {\n"
-        + "    return apply(\n"
-        + "      (String) x);\n"
-        + "  }\n"
-        + "}\n",
+        "new org.apache.calcite.linq4j.function.Function1() {\n"
+            + "  public int apply(String x) {\n"
+            + "    return x.length();\n"
+            + "  }\n"
+            + "  public Object apply(Object x) {\n"
+            + "    return apply(\n"
+            + "      (String) x);\n"
+            + "  }\n"
+            + "}\n",
         Expressions.toString(
             Expressions.lambda(
                 Function1.class,
@@ -334,9 +354,9 @@ public class ExpressionTest {
     // 1-dimensional array with initializer
     assertEquals(
         "new String[] {\n"
-        + "  \"foo\",\n"
-        + "  null,\n"
-        + "  \"bar\\\"baz\"}",
+            + "  \"foo\",\n"
+            + "  null,\n"
+            + "  \"bar\\\"baz\"}",
         Expressions.toString(
             Expressions.newArrayInit(
                 String.class,
@@ -347,12 +367,12 @@ public class ExpressionTest {
     // 2-dimensional array with initializer
     assertEquals(
         "new String[][] {\n"
-        + "  new String[] {\n"
-        + "    \"foo\",\n"
-        + "    \"bar\"},\n"
-        + "  null,\n"
-        + "  new String[] {\n"
-        + "    null}}",
+            + "  new String[] {\n"
+            + "    \"foo\",\n"
+            + "    \"bar\"},\n"
+            + "  null,\n"
+            + "  new String[] {\n"
+            + "    null}}",
         Expressions.toString(
             Expressions.newArrayInit(
                 String.class,
@@ -431,9 +451,9 @@ public class ExpressionTest {
     // array of primitives
     assertEquals(
         "new int[] {\n"
-        + "  1,\n"
-        + "  2,\n"
-        + "  -1}",
+            + "  1,\n"
+            + "  2,\n"
+            + "  -1}",
         Expressions.toString(
             Expressions.constant(new int[]{1, 2, -1})));
 
@@ -498,8 +518,8 @@ public class ExpressionTest {
     // objects and nulls
     assertEquals(
         "new String[] {\n"
-        + "  \"foo\",\n"
-        + "  null}",
+            + "  \"foo\",\n"
+            + "  null}",
         Expressions.toString(
             Expressions.constant(new String[] {"foo", null})));
 
@@ -511,15 +531,15 @@ public class ExpressionTest {
 
     // enum
     assertEquals(
-        "net.hydromatic.linq4j.test.ExpressionTest.MyEnum.X",
+        "org.apache.calcite.linq4j.test.ExpressionTest.MyEnum.X",
         Expressions.toString(
             Expressions.constant(MyEnum.X)));
 
     // array of enum
     assertEquals(
-        "new net.hydromatic.linq4j.test.ExpressionTest.MyEnum[] {\n"
-        + "  net.hydromatic.linq4j.test.ExpressionTest.MyEnum.X,\n"
-        + "  net.hydromatic.linq4j.test.ExpressionTest.MyEnum.Y}",
+        "new org.apache.calcite.linq4j.test.ExpressionTest.MyEnum[] {\n"
+            + "  org.apache.calcite.linq4j.test.ExpressionTest.MyEnum.X,\n"
+            + "  org.apache.calcite.linq4j.test.ExpressionTest.MyEnum.Y}",
         Expressions.toString(
             Expressions.constant(new MyEnum[]{MyEnum.X, MyEnum.Y})));
 
@@ -542,23 +562,23 @@ public class ExpressionTest {
 
     // automatically call constructor if it matches fields
     assertEquals(
-        "new net.hydromatic.linq4j.test.Linq4jTest.Employee[] {\n"
-        + "  new net.hydromatic.linq4j.test.Linq4jTest.Employee(\n"
-        + "    100,\n"
-        + "    \"Fred\",\n"
-        + "    10),\n"
-        + "  new net.hydromatic.linq4j.test.Linq4jTest.Employee(\n"
-        + "    110,\n"
-        + "    \"Bill\",\n"
-        + "    30),\n"
-        + "  new net.hydromatic.linq4j.test.Linq4jTest.Employee(\n"
-        + "    120,\n"
-        + "    \"Eric\",\n"
-        + "    10),\n"
-        + "  new net.hydromatic.linq4j.test.Linq4jTest.Employee(\n"
-        + "    130,\n"
-        + "    \"Janet\",\n"
-        + "    10)}",
+        "new org.apache.calcite.linq4j.test.Linq4jTest.Employee[] {\n"
+            + "  new org.apache.calcite.linq4j.test.Linq4jTest.Employee(\n"
+            + "    100,\n"
+            + "    \"Fred\",\n"
+            + "    10),\n"
+            + "  new org.apache.calcite.linq4j.test.Linq4jTest.Employee(\n"
+            + "    110,\n"
+            + "    \"Bill\",\n"
+            + "    30),\n"
+            + "  new org.apache.calcite.linq4j.test.Linq4jTest.Employee(\n"
+            + "    120,\n"
+            + "    \"Eric\",\n"
+            + "    10),\n"
+            + "  new org.apache.calcite.linq4j.test.Linq4jTest.Employee(\n"
+            + "    130,\n"
+            + "    \"Janet\",\n"
+            + "    10)}",
         Expressions.toString(
             Expressions.constant(Linq4jTest.emps)));
   }
@@ -645,19 +665,19 @@ public class ExpressionTest {
                                         .<Expression>emptyList())))))));
     assertEquals(
         "{\n"
-        + "  final java.util.List<String> baz = java.util.Arrays.asList(\"foo\", \"bar\");\n"
-        + "  new java.util.AbstractList<String>(){\n"
-        + "    public final String qux = \"xyzzy\";\n"
-        + "    public int size() {\n"
-        + "      return baz.size();\n"
-        + "    }\n"
-        + "\n"
-        + "    public String get(int index) {\n"
-        + "      return ((String) baz.get(index)).toUpperCase();\n"
-        + "    }\n"
-        + "\n"
-        + "  };\n"
-        + "}\n",
+            + "  final java.util.List<String> baz = java.util.Arrays.asList(\"foo\", \"bar\");\n"
+            + "  new java.util.AbstractList<String>(){\n"
+            + "    public final String qux = \"xyzzy\";\n"
+            + "    public int size() {\n"
+            + "      return baz.size();\n"
+            + "    }\n"
+            + "\n"
+            + "    public String get(int index) {\n"
+            + "      return ((String) baz.get(index)).toUpperCase();\n"
+            + "    }\n"
+            + "\n"
+            + "  };\n"
+            + "}\n",
         Expressions.toString(e));
   }
 
@@ -682,12 +702,12 @@ public class ExpressionTest {
                     Expressions.preIncrementAssign(yDecl.parameter))));
     assertEquals(
         "{\n"
-        + "  int x = 10;\n"
-        + "  int y = 0;\n"
-        + "  while (x < 5) {\n"
-        + "    ++y;\n"
-        + "  }\n"
-        + "}\n",
+            + "  int x = 10;\n"
+            + "  int y = 0;\n"
+            + "  while (x < 5) {\n"
+            + "    ++y;\n"
+            + "  }\n"
+            + "}\n",
         Expressions.toString(node));
   }
 
@@ -707,21 +727,21 @@ public class ExpressionTest {
                 Expressions.call(
                     Expressions.constant("foo"),
                     "toUpperCase")),
-        Expressions.catch_(cce_,
-            Expressions.return_(null, Expressions.constant(null))),
-        Expressions.catch_(re_,
-            Expressions.throw_(
-                Expressions.new_(IndexOutOfBoundsException.class))));
+            Expressions.catch_(cce_,
+                Expressions.return_(null, Expressions.constant(null))),
+            Expressions.catch_(re_,
+                Expressions.throw_(
+                    Expressions.new_(IndexOutOfBoundsException.class))));
     assertEquals(
         "try {\n"
-        + "  return \"foo\".length();\n"
-        + "} catch (final ClassCastException cce) {\n"
-        + "  return null;\n"
-        + "} catch (RuntimeException re) {\n"
-        + "  throw new IndexOutOfBoundsException();\n"
-        + "} finally {\n"
-        + "  \"foo\".toUpperCase();\n"
-        + "}\n",
+            + "  return \"foo\".length();\n"
+            + "} catch (final ClassCastException cce) {\n"
+            + "  return null;\n"
+            + "} catch (RuntimeException re) {\n"
+            + "  throw new IndexOutOfBoundsException();\n"
+            + "} finally {\n"
+            + "  \"foo\".toUpperCase();\n"
+            + "}\n",
         Expressions.toString(node));
   }
 
@@ -733,24 +753,24 @@ public class ExpressionTest {
     Node node =
         Expressions.ifThen(
             Expressions.constant(true),
-        Expressions.tryFinally(
-            Expressions.block(
-                Expressions.return_(null,
+            Expressions.tryFinally(
+                Expressions.block(
+                    Expressions.return_(null,
+                        Expressions.call(
+                            Expressions.constant("foo"),
+                            "length"))),
+                Expressions.statement(
                     Expressions.call(
                         Expressions.constant("foo"),
-                        "length"))),
-            Expressions.statement(
-                Expressions.call(
-                    Expressions.constant("foo"),
-                    "toUpperCase"))));
+                        "toUpperCase"))));
     assertEquals(
         "if (true) {\n"
-        + "  try {\n"
-        + "    return \"foo\".length();\n"
-        + "  } finally {\n"
-        + "    \"foo\".toUpperCase();\n"
-        + "  }\n"
-        + "}\n",
+            + "  try {\n"
+            + "    return \"foo\".length();\n"
+            + "  } finally {\n"
+            + "    \"foo\".toUpperCase();\n"
+            + "  }\n"
+            + "}\n",
         Expressions.toString(node));
   }
 
@@ -771,12 +791,12 @@ public class ExpressionTest {
                     Expressions.call(re_, "toString"))));
     assertEquals(
         "try {\n"
-        + "  return \"foo\".length();\n"
-        + "} catch (final ClassCastException cce) {\n"
-        + "  return null;\n"
-        + "} catch (RuntimeException re) {\n"
-        + "  return re.toString();\n"
-        + "}\n",
+            + "  return \"foo\".length();\n"
+            + "} catch (final ClassCastException cce) {\n"
+            + "  return null;\n"
+            + "} catch (RuntimeException re) {\n"
+            + "  return re.toString();\n"
+            + "}\n",
         Expressions.toString(node));
   }
 
@@ -837,18 +857,18 @@ public class ExpressionTest {
     checkBlockBuilder(
         false,
         "{\n"
-        + "  final int three = 1 + 2;\n"
-        + "  final int six = three * 2;\n"
-        + "  final int nine = three * three;\n"
-        + "  final int eighteen = three + six + nine;\n"
-        + "  return eighteen;\n"
-        + "}\n");
+            + "  final int three = 1 + 2;\n"
+            + "  final int six = three * 2;\n"
+            + "  final int nine = three * three;\n"
+            + "  final int eighteen = three + six + nine;\n"
+            + "  return eighteen;\n"
+            + "}\n");
     checkBlockBuilder(
         true,
         "{\n"
-        + "  final int three = 1 + 2;\n"
-        + "  return three + three * 2 + three * three;\n"
-        + "}\n");
+            + "  final int three = 1 + 2;\n"
+            + "  return three + three * 2 + three * three;\n"
+            + "}\n");
   }
 
   public void checkBlockBuilder(boolean optimizing, String expected) {
@@ -906,9 +926,9 @@ public class ExpressionTest {
     BlockStatement expression = statements.toBlock();
     assertEquals(
         "{\n"
-        + "  return new java.util.TreeSet(\n"
-        + "      (java.util.Comparator) null).add(null);\n"
-        + "}\n",
+            + "  return new java.util.TreeSet(\n"
+            + "      (java.util.Comparator) null).add(null);\n"
+            + "}\n",
         Expressions.toString(expression));
     expression.accept(new Visitor());
   }
@@ -952,11 +972,11 @@ public class ExpressionTest {
     BlockStatement expression = builder0.toBlock();
     assertEquals(
         "{\n"
-        + "  final int _b = 1 + 2;\n"
-        + "  final int _c = 1 + 3;\n"
-        + "  final int _d = 1 + 4;\n"
-        + "  net.hydromatic.linq4j.test.ExpressionTest.bar(1, _b, _c, _d, net.hydromatic.linq4j.test.ExpressionTest.foo(_c));\n"
-        + "}\n",
+            + "  final int _b = 1 + 2;\n"
+            + "  final int _c = 1 + 3;\n"
+            + "  final int _d = 1 + 4;\n"
+            + "  org.apache.calcite.linq4j.test.ExpressionTest.bar(1, _b, _c, _d, org.apache.calcite.linq4j.test.ExpressionTest.foo(_c));\n"
+            + "}\n",
         Expressions.toString(expression));
     expression.accept(new Visitor());
   }
@@ -970,36 +990,36 @@ public class ExpressionTest {
             (float) 5, (double) 6, (char) 7, true, "string", null
           },
           new AllType(true, (byte) 100, (char) 101, (short) 102, 103,
-            (long) 104, (float) 105, (double) 106, new BigDecimal(107),
-            new BigInteger("108"), "109", null)
+              (long) 104, (float) 105, (double) 106, new BigDecimal(107),
+              new BigInteger("108"), "109", null)
         });
     assertEquals(
         "new Object[] {\n"
-        + "  1,\n"
-        + "  new Object[] {\n"
-        + "    (byte)1,\n"
-        + "    (short)2,\n"
-        + "    3,\n"
-        + "    4L,\n"
-        + "    5.0F,\n"
-        + "    6.0D,\n"
-        + "    (char)7,\n"
-        + "    true,\n"
-        + "    \"string\",\n"
-        + "    null},\n"
-        + "  new net.hydromatic.linq4j.test.ExpressionTest.AllType(\n"
-        + "    true,\n"
-        + "    (byte)100,\n"
-        + "    (char)101,\n"
-        + "    (short)102,\n"
-        + "    103,\n"
-        + "    104L,\n"
-        + "    105.0F,\n"
-        + "    106.0D,\n"
-        + "    new java.math.BigDecimal(107L),\n"
-        + "    new java.math.BigInteger(\"108\"),\n"
-        + "    \"109\",\n"
-        + "    null)}",
+            + "  1,\n"
+            + "  new Object[] {\n"
+            + "    (byte)1,\n"
+            + "    (short)2,\n"
+            + "    3,\n"
+            + "    4L,\n"
+            + "    5.0F,\n"
+            + "    6.0D,\n"
+            + "    (char)7,\n"
+            + "    true,\n"
+            + "    \"string\",\n"
+            + "    null},\n"
+            + "  new org.apache.calcite.linq4j.test.ExpressionTest.AllType(\n"
+            + "    true,\n"
+            + "    (byte)100,\n"
+            + "    (char)101,\n"
+            + "    (short)102,\n"
+            + "    103,\n"
+            + "    104L,\n"
+            + "    105.0F,\n"
+            + "    106.0D,\n"
+            + "    new java.math.BigDecimal(107L),\n"
+            + "    new java.math.BigInteger(\"108\"),\n"
+            + "    \"109\",\n"
+            + "    null)}",
         constant.toString());
     constant.accept(new Visitor());
   }
@@ -1029,12 +1049,12 @@ public class ExpressionTest {
                     Expressions.parameter(int.class, "i"))));
     assertEquals(
         "new Object(){\n"
-        + "  public final String foo = \"bar\";\n"
-        + "  public static class MyClass {\n"
-        + "    int x = 0;\n"
-        + "  }\n"
-        + "  int i;\n"
-        + "}",
+            + "  public final String foo = \"bar\";\n"
+            + "  public static class MyClass {\n"
+            + "    int x = 0;\n"
+            + "  }\n"
+            + "  int i;\n"
+            + "}",
         Expressions.toString(newExpression));
     newExpression.accept(new Visitor());
   }
@@ -1042,10 +1062,10 @@ public class ExpressionTest {
   @Test public void testReturn() {
     assertEquals(
         "if (true) {\n"
-        + "  return;\n"
-        + "} else {\n"
-        + "  return 1;\n"
-        + "}\n",
+            + "  return;\n"
+            + "} else {\n"
+            + "  return 1;\n"
+            + "}\n",
         Expressions.toString(
             Expressions.ifThenElse(
                 Expressions.constant(true),
@@ -1056,12 +1076,12 @@ public class ExpressionTest {
   @Test public void testIfElseIfElse() {
     assertEquals(
         "if (true) {\n"
-        + "  return;\n"
-        + "} else if (false) {\n"
-        + "  return;\n"
-        + "} else {\n"
-        + "  return 1;\n"
-        + "}\n",
+            + "  return;\n"
+            + "} else if (false) {\n"
+            + "  return;\n"
+            + "} else {\n"
+            + "  return 1;\n"
+            + "}\n",
         Expressions.toString(
             Expressions.ifThenElse(
                 Expressions.constant(true),
@@ -1115,10 +1135,10 @@ public class ExpressionTest {
     builder.add(Expressions.return_(null, v6));
     assertEquals(
         "{\n"
-        + "  final Short v = (Short) ((Object[]) p)[4];\n"
-        + "  return (Number) v == null ? (Boolean) null : ("
-        + "(Number) v).intValue() == 1997;\n"
-        + "}\n",
+            + "  final Short v = (Short) ((Object[]) p)[4];\n"
+            + "  return (Number) v == null ? (Boolean) null : ("
+            + "(Number) v).intValue() == 1997;\n"
+            + "}\n",
         Expressions.toString(builder.toBlock()));
   }
 
@@ -1140,10 +1160,10 @@ public class ExpressionTest {
                         i_)))));
     assertEquals(
         "{\n"
-        + "  for (int i = 0; i < 10; i++) {\n"
-        + "    System.out.println(i);\n"
-        + "  }\n"
-        + "}\n",
+            + "  for (int i = 0; i < 10; i++) {\n"
+            + "    System.out.println(i);\n"
+            + "  }\n"
+            + "}\n",
         Expressions.toString(builder.toBlock()));
   }
 
@@ -1168,12 +1188,12 @@ public class ExpressionTest {
                     Expressions.break_(null)))));
     assertEquals(
         "{\n"
-        + "  for (int i = 0, j = 10; ; ) {\n"
-        + "    if (++i < --j) {\n"
-        + "      break;\n"
-        + "    }\n"
-        + "  }\n"
-        + "}\n",
+            + "  for (int i = 0, j = 10; ; ) {\n"
+            + "    if (++i < --j) {\n"
+            + "      break;\n"
+            + "    }\n"
+            + "  }\n"
+            + "}\n",
         Expressions.toString(builder.toBlock()));
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/test/java/org/apache/calcite/linq4j/test/InlinerTest.java
----------------------------------------------------------------------
diff --git a/linq4j/src/test/java/org/apache/calcite/linq4j/test/InlinerTest.java b/linq4j/src/test/java/org/apache/calcite/linq4j/test/InlinerTest.java
index d3cf084..79e2f7d 100644
--- a/linq4j/src/test/java/org/apache/calcite/linq4j/test/InlinerTest.java
+++ b/linq4j/src/test/java/org/apache/calcite/linq4j/test/InlinerTest.java
@@ -14,9 +14,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.test;
+package org.apache.calcite.linq4j.test;
 
-import net.hydromatic.linq4j.expressions.*;
+import org.apache.calcite.linq4j.tree.BlockBuilder;
+import org.apache.calcite.linq4j.tree.DeclarationStatement;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.ExpressionType;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.linq4j.tree.ParameterExpression;
 
 import org.hamcrest.CoreMatchers;
 import org.junit.Before;
@@ -24,7 +29,9 @@ import org.junit.Test;
 
 import java.lang.reflect.Modifier;
 
-import static net.hydromatic.linq4j.test.BlockBuilderBase.*;
+import static org.apache.calcite.linq4j.test.BlockBuilderBase.ONE;
+import static org.apache.calcite.linq4j.test.BlockBuilderBase.TRUE;
+import static org.apache.calcite.linq4j.test.BlockBuilderBase.TWO;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
@@ -78,9 +85,9 @@ public class InlinerTest {
             Expressions.add(decl.parameter, decl.parameter)));
     assertEquals(
         "{\n"
-        + "  final int x = p1 - p2;\n"
-        + "  return x + x;\n"
-        + "}\n",
+            + "  final int x = p1 - p2;\n"
+            + "  return x + x;\n"
+            + "}\n",
         b.toBlock().toString());
   }
 
@@ -102,25 +109,25 @@ public class InlinerTest {
     builder.add(Expressions.return_(null, v));
     assertEquals(
         "{\n"
-        + "  int t;\n"
-        + "  return (t = 1) != a ? t : c;\n"
-        + "}\n",
+            + "  int t;\n"
+            + "  return (t = 1) != a ? t : c;\n"
+            + "}\n",
         Expressions.toString(builder.toBlock()));
   }
 
   @Test public void testAssignInConditionOptimizedOut() {
     checkAssignInConditionOptimizedOut(Modifier.FINAL,
         "{\n"
-        + "  return 1 != a ? b : c;\n"
-        + "}\n");
+            + "  return 1 != a ? b : c;\n"
+            + "}\n");
   }
 
   @Test public void testAssignInConditionNotOptimizedWithoutFinal() {
     checkAssignInConditionOptimizedOut(0,
         "{\n"
-        + "  int t;\n"
-        + "  return (t = 1) != a ? b : c;\n"
-        + "}\n");
+            + "  int t;\n"
+            + "  return (t = 1) != a ? b : c;\n"
+            + "}\n");
   }
 
   void checkAssignInConditionOptimizedOut(int modifiers, String s) {
@@ -162,9 +169,9 @@ public class InlinerTest {
     builder.add(Expressions.return_(null, v));
     assertEquals(
         "{\n"
-        + "  int t = 2;\n"
-        + "  return (t = 1) != a ? t : c;\n"
-        + "}\n",
+            + "  int t = 2;\n"
+            + "  return (t = 1) != a ? t : c;\n"
+            + "}\n",
         Expressions.toString(builder.toBlock()));
   }
 
@@ -183,8 +190,8 @@ public class InlinerTest {
     builder.add(Expressions.return_(null, Expressions.condition(b, t, TWO)));
     assertEquals(
         "{\n"
-        + "  return u + v;\n"
-        + "}\n",
+            + "  return u + v;\n"
+            + "}\n",
         Expressions.toString(builder.toBlock()));
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/test/java/org/apache/calcite/linq4j/test/Linq4jSuite.java
----------------------------------------------------------------------
diff --git a/linq4j/src/test/java/org/apache/calcite/linq4j/test/Linq4jSuite.java b/linq4j/src/test/java/org/apache/calcite/linq4j/test/Linq4jSuite.java
index 91bd886..fd044cb 100644
--- a/linq4j/src/test/java/org/apache/calcite/linq4j/test/Linq4jSuite.java
+++ b/linq4j/src/test/java/org/apache/calcite/linq4j/test/Linq4jSuite.java
@@ -14,10 +14,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.test;
+package org.apache.calcite.linq4j.test;
 
-import net.hydromatic.linq4j.expressions.TypeTest;
-import net.hydromatic.linq4j.function.FunctionTest;
+import org.apache.calcite.linq4j.function.FunctionTest;
+import org.apache.calcite.linq4j.tree.TypeTest;
 
 import org.junit.runner.RunWith;
 import org.junit.runners.Suite;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/test/java/org/apache/calcite/linq4j/test/Linq4jTest.java
----------------------------------------------------------------------
diff --git a/linq4j/src/test/java/org/apache/calcite/linq4j/test/Linq4jTest.java b/linq4j/src/test/java/org/apache/calcite/linq4j/test/Linq4jTest.java
index b9ce961..5c92e9e 100644
--- a/linq4j/src/test/java/org/apache/calcite/linq4j/test/Linq4jTest.java
+++ b/linq4j/src/test/java/org/apache/calcite/linq4j/test/Linq4jTest.java
@@ -14,20 +14,53 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.test;
-
-import net.hydromatic.linq4j.*;
-import net.hydromatic.linq4j.expressions.*;
-import net.hydromatic.linq4j.function.*;
+package org.apache.calcite.linq4j.test;
+
+import org.apache.calcite.linq4j.AbstractEnumerable;
+import org.apache.calcite.linq4j.Enumerable;
+import org.apache.calcite.linq4j.EnumerableDefaults;
+import org.apache.calcite.linq4j.Enumerator;
+import org.apache.calcite.linq4j.ExtendedEnumerable;
+import org.apache.calcite.linq4j.Grouping;
+import org.apache.calcite.linq4j.Linq4j;
+import org.apache.calcite.linq4j.Lookup;
+import org.apache.calcite.linq4j.Queryable;
+import org.apache.calcite.linq4j.QueryableDefaults;
+import org.apache.calcite.linq4j.function.EqualityComparer;
+import org.apache.calcite.linq4j.function.Function0;
+import org.apache.calcite.linq4j.function.Function1;
+import org.apache.calcite.linq4j.function.Function2;
+import org.apache.calcite.linq4j.function.Functions;
+import org.apache.calcite.linq4j.function.IntegerFunction1;
+import org.apache.calcite.linq4j.function.Predicate1;
+import org.apache.calcite.linq4j.function.Predicate2;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.linq4j.tree.ParameterExpression;
 
 import com.example.Linq4jExample;
 
 import org.junit.Test;
 
-import java.util.*;
-
-import static org.hamcrest.CoreMatchers.*;
-import static org.junit.Assert.*;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.TreeSet;
+
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.CoreMatchers.nullValue;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 /**
  * Tests for LINQ4J.
@@ -369,16 +402,16 @@ public class Linq4jTest {
 
   @Test public void testContainsWithEqualityComparer() {
     EqualityComparer<Employee> compareByEmpno =
-            new EqualityComparer<Employee>() {
-        public boolean equal(Employee e1, Employee e2) {
-          return e1 != null && e2 != null
-                  && e1.empno == e2.empno;
-        }
+        new EqualityComparer<Employee>() {
+          public boolean equal(Employee e1, Employee e2) {
+            return e1 != null && e2 != null
+                && e1.empno == e2.empno;
+          }
 
-        public int hashCode(Employee t) {
-          return t == null ? 0x789d : t.hashCode();
-        }
-      };
+          public int hashCode(Employee t) {
+            return t == null ? 0x789d : t.hashCode();
+          }
+        };
 
     Employee e = emps[1];
     Employee employeeClone = new Employee(e.empno, e.name, e.deptno);
@@ -386,11 +419,11 @@ public class Linq4jTest {
 
     assertEquals(e, employeeClone);
     assertTrue(Linq4j.asEnumerable(emps)
-            .contains(e, compareByEmpno));
+        .contains(e, compareByEmpno));
     assertTrue(Linq4j.asEnumerable(emps)
-            .contains(employeeClone, compareByEmpno));
+        .contains(employeeClone, compareByEmpno));
     assertFalse(Linq4j.asEnumerable(emps)
-            .contains(employeeOther, compareByEmpno));
+        .contains(employeeOther, compareByEmpno));
 
   }
 
@@ -539,7 +572,7 @@ public class Linq4jTest {
     }
     assertEquals(
         "Fred: [Employee(name: Fred, deptno:10), Employee(name: Bill, deptno:30), Employee(name: Eric, deptno:10)]\n"
-        + "Janet: [Employee(name: Janet, deptno:10)]\n",
+            + "Janet: [Employee(name: Janet, deptno:10)]\n",
         buf.toString());
   }
 
@@ -818,8 +851,8 @@ public class Linq4jTest {
             .toString();
     assertEquals(
         "[[Fred, Eric, Janet] work(s) in Sales, "
-        + "[] work(s) in HR, "
-        + "[Bill] work(s) in Marketing]",
+            + "[] work(s) in HR, "
+            + "[Bill] work(s) in Marketing]",
         s);
   }
 
@@ -844,9 +877,9 @@ public class Linq4jTest {
             .toString();
     assertEquals(
         "[Bill works in Marketing, "
-        + "Eric works in Sales, "
-        + "Fred works in Sales, "
-        + "Janet works in Sales]",
+            + "Eric works in Sales, "
+            + "Fred works in Sales, "
+            + "Janet works in Sales]",
         s);
   }
 
@@ -873,10 +906,10 @@ public class Linq4jTest {
             .toString();
     assertEquals(
         "[Bill works in Marketing, "
-        + "Cedric works in null, "
-        + "Eric works in Sales, "
-        + "Fred works in Sales, "
-        + "Janet works in Sales]",
+            + "Cedric works in null, "
+            + "Eric works in Sales, "
+            + "Fred works in Sales, "
+            + "Janet works in Sales]",
         s);
   }
 
@@ -903,10 +936,10 @@ public class Linq4jTest {
             .toString();
     assertEquals(
         "[Bill works in Marketing, "
-        + "Eric works in Sales, "
-        + "Fred works in Sales, "
-        + "Janet works in Sales, "
-        + "null works in HR]",
+            + "Eric works in Sales, "
+            + "Fred works in Sales, "
+            + "Janet works in Sales, "
+            + "null works in HR]",
         s);
   }
 
@@ -933,11 +966,11 @@ public class Linq4jTest {
             .toString();
     assertEquals(
         "[Bill works in Marketing, "
-        + "Cedric works in null, "
-        + "Eric works in Sales, "
-        + "Fred works in Sales, "
-        + "Janet works in Sales, "
-        + "null works in HR]",
+            + "Cedric works in null, "
+            + "Eric works in Sales, "
+            + "Fred works in Sales, "
+            + "Janet works in Sales, "
+            + "null works in HR]",
         s);
   }
 
@@ -1173,7 +1206,7 @@ public class Linq4jTest {
               public boolean apply(Department v1, Integer v2) {
                 // Make sure we're passed the correct indices
                 assertEquals(
-                  "Invalid index passed to function", index++, (int) v2);
+                    "Invalid index passed to function", index++, (int) v2);
                 return 20 != v1.deptno;
               }
             }).toList();
@@ -1272,7 +1305,7 @@ public class Linq4jTest {
             new Predicate2<Department, Integer>() {
               public boolean apply(Department v1, Integer v2) {
                 return v1.name.equals("Sales")
-                       || v2 == 1;
+                    || v2 == 1;
               }
             }).count());
 
@@ -1301,7 +1334,7 @@ public class Linq4jTest {
                 new Predicate2<Department, Integer>() {
                   public boolean apply(Department v1, Integer v2) {
                     return v1.name.equals("Sales")
-                           || v2 == 1;
+                        || v2 == 1;
                   }
                 })).count());
   }
@@ -1310,9 +1343,9 @@ public class Linq4jTest {
     // Note: sort is stable. Records occur Fred, Eric, Janet in input.
     assertEquals(
         "[Employee(name: Fred, deptno:10),"
-        + " Employee(name: Eric, deptno:10),"
-        + " Employee(name: Janet, deptno:10),"
-        + " Employee(name: Bill, deptno:30)]",
+            + " Employee(name: Eric, deptno:10),"
+            + " Employee(name: Janet, deptno:10),"
+            + " Employee(name: Bill, deptno:30)]",
         Linq4j.asEnumerable(emps).orderBy(EMP_DEPTNO_SELECTOR)
             .toList().toString());
   }
@@ -1320,9 +1353,9 @@ public class Linq4jTest {
   @Test public void testOrderByComparator() {
     assertEquals(
         "[Employee(name: Bill, deptno:30),"
-        + " Employee(name: Eric, deptno:10),"
-        + " Employee(name: Fred, deptno:10),"
-        + " Employee(name: Janet, deptno:10)]",
+            + " Employee(name: Eric, deptno:10),"
+            + " Employee(name: Fred, deptno:10),"
+            + " Employee(name: Janet, deptno:10)]",
         Linq4j.asEnumerable(emps)
             .orderBy(EMP_NAME_SELECTOR)
             .orderBy(
@@ -1334,9 +1367,9 @@ public class Linq4jTest {
     // OrderBy in series works because sort is stable.
     assertEquals(
         "[Employee(name: Eric, deptno:10),"
-        + " Employee(name: Fred, deptno:10),"
-        + " Employee(name: Janet, deptno:10),"
-        + " Employee(name: Bill, deptno:30)]",
+            + " Employee(name: Fred, deptno:10),"
+            + " Employee(name: Janet, deptno:10),"
+            + " Employee(name: Bill, deptno:30)]",
         Linq4j.asEnumerable(emps)
             .orderBy(EMP_NAME_SELECTOR)
             .orderBy(EMP_DEPTNO_SELECTOR)
@@ -1346,9 +1379,9 @@ public class Linq4jTest {
   @Test public void testOrderByDescending() {
     assertEquals(
         "[Employee(name: Janet, deptno:10),"
-        + " Employee(name: Fred, deptno:10),"
-        + " Employee(name: Eric, deptno:10),"
-        + " Employee(name: Bill, deptno:30)]",
+            + " Employee(name: Fred, deptno:10),"
+            + " Employee(name: Eric, deptno:10),"
+            + " Employee(name: Bill, deptno:30)]",
         Linq4j.asEnumerable(emps)
             .orderByDescending(EMP_NAME_SELECTOR)
             .toList().toString());
@@ -1357,9 +1390,9 @@ public class Linq4jTest {
   @Test public void testReverse() {
     assertEquals(
         "[Employee(name: Janet, deptno:10),"
-        + " Employee(name: Eric, deptno:10),"
-        + " Employee(name: Bill, deptno:30),"
-        + " Employee(name: Fred, deptno:10)]",
+            + " Employee(name: Eric, deptno:10),"
+            + " Employee(name: Bill, deptno:30),"
+            + " Employee(name: Fred, deptno:10)]",
         Linq4j.asEnumerable(emps)
             .reverse()
             .toList()
@@ -1430,8 +1463,7 @@ public class Linq4jTest {
       return "Employee(name: " + name + ", deptno:" + deptno + ")";
     }
 
-    @Override
-    public int hashCode() {
+    @Override public int hashCode() {
       final int prime = 31;
       int result = 1;
       result = prime * result + deptno;
@@ -1440,8 +1472,7 @@ public class Linq4jTest {
       return result;
     }
 
-    @Override
-    public boolean equals(Object obj) {
+    @Override public boolean equals(Object obj) {
       if (this == obj) {
         return true;
       }
@@ -1483,9 +1514,9 @@ public class Linq4jTest {
 
     public String toString() {
       return "Department(name: " + name
-             + ", deptno:" + deptno
-             + ", employees: " + employees
-             + ")";
+          + ", deptno:" + deptno
+          + ", employees: " + employees
+          + ")";
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/test/java/org/apache/calcite/linq4j/test/OptimizerTest.java
----------------------------------------------------------------------
diff --git a/linq4j/src/test/java/org/apache/calcite/linq4j/test/OptimizerTest.java b/linq4j/src/test/java/org/apache/calcite/linq4j/test/OptimizerTest.java
index ab2b7b0..dd0d82d 100644
--- a/linq4j/src/test/java/org/apache/calcite/linq4j/test/OptimizerTest.java
+++ b/linq4j/src/test/java/org/apache/calcite/linq4j/test/OptimizerTest.java
@@ -14,23 +14,37 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.test;
+package org.apache.calcite.linq4j.test;
 
-import net.hydromatic.linq4j.Linq4j;
-import net.hydromatic.linq4j.expressions.*;
+import org.apache.calcite.linq4j.Linq4j;
+import org.apache.calcite.linq4j.tree.ConstantExpression;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.linq4j.tree.ParameterExpression;
 
 import org.junit.Test;
 
 import java.lang.reflect.Method;
 import java.lang.reflect.Modifier;
 
-import static net.hydromatic.linq4j.test.BlockBuilderBase.*;
+import static org.apache.calcite.linq4j.test.BlockBuilderBase.FALSE;
+import static org.apache.calcite.linq4j.test.BlockBuilderBase.FOUR;
+import static org.apache.calcite.linq4j.test.BlockBuilderBase.NULL;
+import static org.apache.calcite.linq4j.test.BlockBuilderBase.NULL_INTEGER;
+import static org.apache.calcite.linq4j.test.BlockBuilderBase.ONE;
+import static org.apache.calcite.linq4j.test.BlockBuilderBase.THREE;
+import static org.apache.calcite.linq4j.test.BlockBuilderBase.TRUE;
+import static org.apache.calcite.linq4j.test.BlockBuilderBase.TRUE_B;
+import static org.apache.calcite.linq4j.test.BlockBuilderBase.TWO;
+import static org.apache.calcite.linq4j.test.BlockBuilderBase.bool;
+import static org.apache.calcite.linq4j.test.BlockBuilderBase.optimize;
 
 import static org.hamcrest.CoreMatchers.equalTo;
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
 
 /**
- * Unit test for {@link net.hydromatic.linq4j.expressions.BlockBuilder}
+ * Unit test for {@link org.apache.calcite.linq4j.tree.BlockBuilder}
  * optimization capabilities.
  */
 public class OptimizerTest {
@@ -110,7 +124,7 @@ public class OptimizerTest {
         optimize(Expressions.condition(
             Expressions.parameter(boolean.class, "a"),
             TRUE_B, Expressions.call(Boolean.class, "valueOf",
-            Expressions.parameter(boolean.class, "b")))));
+                Expressions.parameter(boolean.class, "b")))));
   }
 
   @Test public void testOptimizeTernaryABtrue() {
@@ -141,8 +155,8 @@ public class OptimizerTest {
     // (v ? (Integer) null : inp0_) == null
     assertEquals("{\n  return v || inp0_ == null;\n}\n",
         optimize(Expressions.equal(Expressions.condition(
-            Expressions.parameter(boolean.class, "v"),
-            NULL_INTEGER, Expressions.parameter(Integer.class, "inp0_")),
+                Expressions.parameter(boolean.class, "v"),
+                NULL_INTEGER, Expressions.parameter(Integer.class, "inp0_")),
             NULL)));
   }
 
@@ -150,8 +164,8 @@ public class OptimizerTest {
     // (v ? inp0_ : (Integer) null) == null
     assertEquals("{\n  return !v || inp0_ == null;\n}\n",
         optimize(Expressions.equal(Expressions.condition(
-            Expressions.parameter(boolean.class, "v"),
-            Expressions.parameter(Integer.class, "inp0_"), NULL_INTEGER),
+                Expressions.parameter(boolean.class, "v"),
+                Expressions.parameter(Integer.class, "inp0_"), NULL_INTEGER),
             NULL)));
   }
 
@@ -175,8 +189,8 @@ public class OptimizerTest {
     // (v ? (Integer) null : inp0_) != null
     assertEquals("{\n  return !(v || inp0_ == null);\n}\n",
         optimize(Expressions.notEqual(Expressions.condition(
-            Expressions.parameter(boolean.class, "v"),
-            NULL_INTEGER, Expressions.parameter(Integer.class, "inp0_")),
+                Expressions.parameter(boolean.class, "v"),
+                NULL_INTEGER, Expressions.parameter(Integer.class, "inp0_")),
             NULL)));
   }
 
@@ -184,8 +198,8 @@ public class OptimizerTest {
     // (v ? inp0_ : (Integer) null) != null
     assertEquals("{\n  return !(!v || inp0_ == null);\n}\n",
         optimize(Expressions.notEqual(Expressions.condition(
-            Expressions.parameter(boolean.class, "v"),
-            Expressions.parameter(Integer.class, "inp0_"), NULL_INTEGER),
+                Expressions.parameter(boolean.class, "v"),
+                Expressions.parameter(Integer.class, "inp0_"), NULL_INTEGER),
             NULL)));
   }
 
@@ -487,12 +501,12 @@ public class OptimizerTest {
     Expression bool = Expressions.parameter(boolean.class, "bool");
     assertEquals(
         "{\n"
-        + "  if (bool) {\n"
-        + "    return 1;\n"
-        + "  } else {\n"
-        + "    return 2;\n"
-        + "  }\n"
-        + "}\n",
+            + "  if (bool) {\n"
+            + "    return 1;\n"
+            + "  } else {\n"
+            + "    return 2;\n"
+            + "  }\n"
+            + "}\n",
         optimize(
             Expressions.ifThenElse(bool,
                 Expressions.return_(null, ONE),
@@ -505,12 +519,12 @@ public class OptimizerTest {
     Expression bool = Expressions.parameter(boolean.class, "bool");
     assertEquals(
         "{\n"
-        + "  if (bool) {\n"
-        + "    return 1;\n"
-        + "  } else {\n"
-        + "    return 2;\n"
-        + "  }\n"
-        + "}\n",
+            + "  if (bool) {\n"
+            + "    return 1;\n"
+            + "  } else {\n"
+            + "    return 2;\n"
+            + "  }\n"
+            + "}\n",
         optimize(
             Expressions.ifThenElse(bool,
                 Expressions.return_(null, ONE),
@@ -524,10 +538,10 @@ public class OptimizerTest {
     Expression bool = Expressions.parameter(boolean.class, "bool");
     assertEquals(
         "{\n"
-        + "  if (bool) {\n"
-        + "    return 1;\n"
-        + "  }\n"
-        + "}\n",
+            + "  if (bool) {\n"
+            + "    return 1;\n"
+            + "  }\n"
+            + "}\n",
         optimize(
             Expressions.ifThenElse(bool,
                 Expressions.return_(null, ONE),
@@ -540,12 +554,12 @@ public class OptimizerTest {
     Expression bool = Expressions.parameter(boolean.class, "bool");
     assertEquals(
         "{\n"
-        + "  if (bool) {\n"
-        + "    return 1;\n"
-        + "  } else {\n"
-        + "    return 3;\n"
-        + "  }\n"
-        + "}\n",
+            + "  if (bool) {\n"
+            + "    return 1;\n"
+            + "  } else {\n"
+            + "    return 3;\n"
+            + "  }\n"
+            + "}\n",
         optimize(
             Expressions.ifThenElse(bool,
                 Expressions.return_(null, ONE),
@@ -559,12 +573,12 @@ public class OptimizerTest {
     Expression bool = Expressions.parameter(boolean.class, "bool");
     assertEquals(
         "{\n"
-        + "  if (bool) {\n"
-        + "    return 1;\n"
-        + "  } else {\n"
-        + "    return 4;\n"
-        + "  }\n"
-        + "}\n",
+            + "  if (bool) {\n"
+            + "    return 1;\n"
+            + "  } else {\n"
+            + "    return 4;\n"
+            + "  }\n"
+            + "}\n",
         optimize(
             Expressions.ifThenElse(bool,
                 Expressions.return_(null, ONE),
@@ -697,8 +711,7 @@ public class OptimizerTest {
                         Expressions.field(null, System.class, "out"),
                         "println",
                         x_)))),
-        equalTo(
-            "{\n"
+        equalTo("{\n"
             + "  long x = 0L;\n"
             + "  if (System.nanoTime() > 0L) {\n"
             + "    x = System.currentTimeMillis();\n"
@@ -727,8 +740,7 @@ public class OptimizerTest {
                 Expressions.ifThen(
                     Expressions.greaterThan(Expressions.call(mT), zero),
                     Expressions.statement(Expressions.assign(x_, y_))))),
-        equalTo(
-            "{\n"
+        equalTo("{\n"
             + "  long x = 0L;\n"
             + "  if (System.currentTimeMillis() > 0L) {\n"
             + "    x = System.currentTimeMillis();\n"

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/test/java/org/apache/calcite/linq4j/test/PrimitiveTest.java
----------------------------------------------------------------------
diff --git a/linq4j/src/test/java/org/apache/calcite/linq4j/test/PrimitiveTest.java b/linq4j/src/test/java/org/apache/calcite/linq4j/test/PrimitiveTest.java
index b341b5c..841fa40 100644
--- a/linq4j/src/test/java/org/apache/calcite/linq4j/test/PrimitiveTest.java
+++ b/linq4j/src/test/java/org/apache/calcite/linq4j/test/PrimitiveTest.java
@@ -14,16 +14,22 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.test;
+package org.apache.calcite.linq4j.test;
 
-import net.hydromatic.linq4j.expressions.Primitive;
+import org.apache.calcite.linq4j.tree.Primitive;
 
 import org.junit.Test;
 
 import java.util.ArrayList;
 import java.util.List;
 
-import static org.junit.Assert.*;
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 /**
  * Unit test for {@link Primitive}.
@@ -118,7 +124,8 @@ public class PrimitiveTest {
     }
   }
 
-  /** Test for {@link Primitive#send(net.hydromatic.linq4j.expressions.Primitive.Source, net.hydromatic.linq4j.expressions.Primitive.Sink)}. */
+  /** Test for
+   * {@link Primitive#send(org.apache.calcite.linq4j.tree.Primitive.Source, org.apache.calcite.linq4j.tree.Primitive.Sink)}. */
   @Test public void testSendSource() {
     final List<Object> list = new ArrayList<Object>();
     for (Primitive primitive : Primitive.values()) {
@@ -216,18 +223,17 @@ public class PrimitiveTest {
             }
           });
     }
-    assertEquals(
-        "[boolean, boolean, true, "
-        + "byte, byte, 0, "
-        + "char, char, \u0000, "
-        + "short, short, 0, "
-        + "int, int, 0, "
-        + "long, long, 0, "
-        + "float, float, 0.0, "
-        + "double, double, 0.0, "
-        + "class java.lang.Object, class java.lang.Object, 0, "
-        + "class java.lang.Object, class java.lang.Object, 0]",
-        list.toString());
+    assertThat(list.toString(),
+        equalTo("[boolean, boolean, true, "
+            + "byte, byte, 0, "
+            + "char, char, \u0000, "
+            + "short, short, 0, "
+            + "int, int, 0, "
+            + "long, long, 0, "
+            + "float, float, 0.0, "
+            + "double, double, 0.0, "
+            + "class java.lang.Object, class java.lang.Object, 0, "
+            + "class java.lang.Object, class java.lang.Object, 0]"));
   }
 
   /** Test for {@link Primitive#permute(Object, int[])}. */

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/test/java/org/apache/calcite/linq4j/test/package-info.java
----------------------------------------------------------------------
diff --git a/linq4j/src/test/java/org/apache/calcite/linq4j/test/package-info.java b/linq4j/src/test/java/org/apache/calcite/linq4j/test/package-info.java
index 2122b1e..9745b87 100644
--- a/linq4j/src/test/java/org/apache/calcite/linq4j/test/package-info.java
+++ b/linq4j/src/test/java/org/apache/calcite/linq4j/test/package-info.java
@@ -18,6 +18,6 @@
 /**
  * Core linq4j tests.
  */
-package net.hydromatic.linq4j.test;
+package org.apache.calcite.linq4j.test;
 
 // End package-info.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/test/java/org/apache/calcite/linq4j/tree/TypeTest.java
----------------------------------------------------------------------
diff --git a/linq4j/src/test/java/org/apache/calcite/linq4j/tree/TypeTest.java b/linq4j/src/test/java/org/apache/calcite/linq4j/tree/TypeTest.java
index 8cd5c8f..42b3fd2 100644
--- a/linq4j/src/test/java/org/apache/calcite/linq4j/tree/TypeTest.java
+++ b/linq4j/src/test/java/org/apache/calcite/linq4j/tree/TypeTest.java
@@ -14,11 +14,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.expressions;
+package org.apache.calcite.linq4j.tree;
 
 import org.junit.Test;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
 
 /**
  * Test for {@link Types#gcd}.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/test/java/org/apache/calcite/linq4j/tree/package-info.java
----------------------------------------------------------------------
diff --git a/linq4j/src/test/java/org/apache/calcite/linq4j/tree/package-info.java b/linq4j/src/test/java/org/apache/calcite/linq4j/tree/package-info.java
index 2f94472..655a4f5 100644
--- a/linq4j/src/test/java/org/apache/calcite/linq4j/tree/package-info.java
+++ b/linq4j/src/test/java/org/apache/calcite/linq4j/tree/package-info.java
@@ -18,6 +18,6 @@
 /**
  * Tests for expressions.
  */
-package net.hydromatic.linq4j.expressions;
+package org.apache.calcite.linq4j.tree;
 
 // End package-info.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/mongodb/pom.xml
----------------------------------------------------------------------
diff --git a/mongodb/pom.xml b/mongodb/pom.xml
index 55e7009..11980cb 100644
--- a/mongodb/pom.xml
+++ b/mongodb/pom.xml
@@ -104,32 +104,13 @@ limitations under the License.
         <artifactId>maven-surefire-plugin</artifactId>
         <configuration>
           <includes>
-            <include>net/hydromatic/optiq/test/MongoAdapterTest.java</include>
+            <include>org/apache/calcite/test/MongoAdapterTest.java</include>
           </includes>
           <threadCount>6</threadCount>
           <parallel>both</parallel>
           <argLine>-Xmx1024m</argLine>
         </configuration>
       </plugin>
-      <plugin>
-        <groupId>org.codehaus.mojo</groupId>
-        <artifactId>javacc-maven-plugin</artifactId>
-        <executions>
-          <execution>
-            <id>javacc</id>
-            <goals>
-              <goal>javacc</goal>
-            </goals>
-            <configuration>
-              <includes>
-                <include>**/CombinedParser.jj</include>
-              </includes>
-              <lookAhead>2</lookAhead>
-              <isStatic>false</isStatic>
-            </configuration>
-          </execution>
-        </executions>
-      </plugin>
     </plugins>
   </build>
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoAggregate.java
----------------------------------------------------------------------
diff --git a/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoAggregate.java b/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoAggregate.java
index 7346d32..9a1744e 100644
--- a/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoAggregate.java
+++ b/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoAggregate.java
@@ -14,27 +14,35 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl.mongodb;
+package org.apache.calcite.adapter.mongodb;
 
-import net.hydromatic.optiq.util.BitSets;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.InvalidRelException;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Aggregate;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.sql.SqlAggFunction;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.fun.SqlSumAggFunction;
+import org.apache.calcite.sql.fun.SqlSumEmptyIsZeroAggFunction;
+import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.Util;
 
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.RelOptCluster;
-import org.eigenbase.relopt.RelTraitSet;
-import org.eigenbase.sql.fun.SqlStdOperatorTable;
-import org.eigenbase.sql.fun.SqlSumAggFunction;
-import org.eigenbase.sql.fun.SqlSumEmptyIsZeroAggFunction;
-import org.eigenbase.util.Util;
-
-import java.util.*;
+import java.util.AbstractList;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.List;
 
 /**
- * Implementation of {@link AggregateRelBase} relational expression in MongoDB.
+ * Implementation of
+ * {@link org.apache.calcite.rel.core.Aggregate} relational expression
+ * in MongoDB.
  */
-public class MongoAggregateRel
-    extends AggregateRelBase
+public class MongoAggregate
+    extends Aggregate
     implements MongoRel {
-  public MongoAggregateRel(
+  public MongoAggregate(
       RelOptCluster cluster,
       RelTraitSet traitSet,
       RelNode child,
@@ -53,10 +61,10 @@ public class MongoAggregateRel
     }
   }
 
-  @Override public AggregateRelBase copy(RelTraitSet traitSet, RelNode input,
+  @Override public Aggregate copy(RelTraitSet traitSet, RelNode input,
       BitSet groupSet, List<AggregateCall> aggCalls) {
     try {
-      return new MongoAggregateRel(getCluster(), traitSet, input, groupSet,
+      return new MongoAggregate(getCluster(), traitSet, input, groupSet,
           aggCalls);
     } catch (InvalidRelException e) {
       // Semantic error not possible. Must be a bug. Convert to
@@ -66,10 +74,10 @@ public class MongoAggregateRel
   }
 
   public void implement(Implementor implementor) {
-    implementor.visitChild(0, getChild());
+    implementor.visitChild(0, getInput());
     List<String> list = new ArrayList<String>();
     final List<String> inNames =
-        MongoRules.mongoFieldNames(getChild().getRowType());
+        MongoRules.mongoFieldNames(getInput().getRowType());
     final List<String> outNames = MongoRules.mongoFieldNames(getRowType());
     int i = 0;
     if (groupSet.cardinality() == 1) {
@@ -97,8 +105,7 @@ public class MongoAggregateRel
       fixups = new AbstractList<String>() {
         @Override public String get(int index) {
           final String outName = outNames.get(index);
-          return MongoRules.maybeQuote(outName)
-              + ": "
+          return MongoRules.maybeQuote(outName) + ": "
               + MongoRules.maybeQuote("$" + (index == 0 ? "_id" : outName));
         }
 
@@ -130,7 +137,7 @@ public class MongoAggregateRel
     }
   }
 
-  private String toMongo(Aggregation aggregation, List<String> inNames,
+  private String toMongo(SqlAggFunction aggregation, List<String> inNames,
       List<Integer> args) {
     if (aggregation == SqlStdOperatorTable.COUNT) {
       if (args.size() == 0) {
@@ -161,4 +168,4 @@ public class MongoAggregateRel
   }
 }
 
-// End MongoAggregateRel.java
+// End MongoAggregate.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoEnumerator.java
----------------------------------------------------------------------
diff --git a/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoEnumerator.java b/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoEnumerator.java
index c2d028c..fdb7112 100644
--- a/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoEnumerator.java
+++ b/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoEnumerator.java
@@ -14,16 +14,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl.mongodb;
+package org.apache.calcite.adapter.mongodb;
 
-import net.hydromatic.linq4j.Enumerator;
-import net.hydromatic.linq4j.function.Function1;
+import org.apache.calcite.linq4j.Enumerator;
+import org.apache.calcite.linq4j.function.Function1;
+import org.apache.calcite.util.DateTimeUtil;
 
-import org.eigenbase.util14.DateTimeUtil;
+import com.mongodb.DBCursor;
+import com.mongodb.DBObject;
 
-import com.mongodb.*;
-
-import java.util.*;
+import java.util.Date;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
 
 /** Enumerator that reads from a MongoDB collection. */
 class MongoEnumerator implements Enumerator<Object> {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoFilter.java
----------------------------------------------------------------------
diff --git a/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoFilter.java b/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoFilter.java
index e73bc54..5f717d7 100644
--- a/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoFilter.java
+++ b/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoFilter.java
@@ -14,27 +14,37 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl.mongodb;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.rex.*;
-import org.eigenbase.util.JsonBuilder;
-import org.eigenbase.util.Pair;
+package org.apache.calcite.adapter.mongodb;
+
+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.RelNode;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.util.JsonBuilder;
+import org.apache.calcite.util.Pair;
 
 import com.google.common.collect.HashMultimap;
 import com.google.common.collect.Multimap;
 
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
 
 /**
- * Implementation of a {@link org.eigenbase.rel.FilterRel} relational expression
- * in MongoDB.
+ * Implementation of a {@link org.apache.calcite.rel.core.Filter}
+ * relational expression in MongoDB.
  */
-public class MongoFilterRel
-    extends FilterRelBase
-    implements MongoRel {
-  public MongoFilterRel(
+public class MongoFilter extends Filter implements MongoRel {
+  public MongoFilter(
       RelOptCluster cluster,
       RelTraitSet traitSet,
       RelNode child,
@@ -44,18 +54,17 @@ public class MongoFilterRel
     assert getConvention() == child.getConvention();
   }
 
-  @Override
-  public RelOptCost computeSelfCost(RelOptPlanner planner) {
+  @Override public RelOptCost computeSelfCost(RelOptPlanner planner) {
     return super.computeSelfCost(planner).multiplyBy(0.1);
   }
 
-  public MongoFilterRel copy(RelTraitSet traitSet, RelNode input,
+  public MongoFilter copy(RelTraitSet traitSet, RelNode input,
       RexNode condition) {
-    return new MongoFilterRel(getCluster(), traitSet, input, condition);
+    return new MongoFilter(getCluster(), traitSet, input, condition);
   }
 
   public void implement(Implementor implementor) {
-    implementor.visitChild(0, getChild());
+    implementor.visitChild(0, getInput());
     Translator translator =
         new Translator(MongoRules.mongoFieldNames(getRowType()));
     String match = translator.translateMatch(condition);
@@ -202,4 +211,4 @@ public class MongoFilterRel
   }
 }
 
-// End MongoFilterRel.java
+// End MongoFilter.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoMethod.java
----------------------------------------------------------------------
diff --git a/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoMethod.java b/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoMethod.java
index ec03fbe..470b203 100644
--- a/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoMethod.java
+++ b/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoMethod.java
@@ -14,14 +14,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl.mongodb;
+package org.apache.calcite.adapter.mongodb;
 
-import net.hydromatic.linq4j.expressions.Types;
+import org.apache.calcite.linq4j.tree.Types;
 
 import com.google.common.collect.ImmutableMap;
 
 import java.lang.reflect.Method;
-import java.util.*;
+import java.util.List;
 
 /**
  * Builtin methods in the MongoDB adapter.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoProject.java
----------------------------------------------------------------------
diff --git a/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoProject.java b/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoProject.java
index 0b1521a..9bbf512 100644
--- a/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoProject.java
+++ b/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoProject.java
@@ -14,34 +14,38 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl.mongodb;
+package org.apache.calcite.adapter.mongodb;
 
-import net.hydromatic.optiq.impl.java.JavaTypeFactory;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
 
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.rex.*;
-import org.eigenbase.util.Pair;
-import org.eigenbase.util.Util;
-
-import java.util.*;
+import java.util.ArrayList;
+import java.util.List;
 
 /**
- * Implementation of {@link ProjectRel} relational expression in
- * MongoDB.
+ * Implementation of {@link org.apache.calcite.rel.core.Project}
+ * relational expression in MongoDB.
  */
-public class MongoProjectRel extends ProjectRelBase implements MongoRel {
-  public MongoProjectRel(RelOptCluster cluster, RelTraitSet traitSet,
+public class MongoProject extends Project implements MongoRel {
+  public MongoProject(RelOptCluster cluster, RelTraitSet traitSet,
       RelNode child, List<RexNode> exps, RelDataType rowType, int flags) {
     super(cluster, traitSet, child, exps, rowType, flags);
     assert getConvention() == MongoRel.CONVENTION;
     assert getConvention() == child.getConvention();
   }
 
-  @Override public ProjectRelBase copy(RelTraitSet traitSet, RelNode input,
+  @Override public Project copy(RelTraitSet traitSet, RelNode input,
       List<RexNode> exps, RelDataType rowType) {
-    return new MongoProjectRel(getCluster(), traitSet, input, exps,
+    return new MongoProject(getCluster(), traitSet, input, exps,
         rowType, flags);
   }
 
@@ -50,12 +54,12 @@ public class MongoProjectRel extends ProjectRelBase implements MongoRel {
   }
 
   public void implement(Implementor implementor) {
-    implementor.visitChild(0, getChild());
+    implementor.visitChild(0, getInput());
 
     final MongoRules.RexToMongoTranslator translator =
         new MongoRules.RexToMongoTranslator(
             (JavaTypeFactory) getCluster().getTypeFactory(),
-            MongoRules.mongoFieldNames(getChild().getRowType()));
+            MongoRules.mongoFieldNames(getInput().getRowType()));
     final List<String> items = new ArrayList<String>();
     for (Pair<RexNode, String> pair : getNamedProjects()) {
       final String name = pair.right;
@@ -71,4 +75,4 @@ public class MongoProjectRel extends ProjectRelBase implements MongoRel {
   }
 }
 
-// End MongoProjectRel.java
+// End MongoProject.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoRel.java
----------------------------------------------------------------------
diff --git a/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoRel.java b/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoRel.java
index 272123d..c05c965 100644
--- a/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoRel.java
+++ b/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoRel.java
@@ -14,12 +14,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl.mongodb;
+package org.apache.calcite.adapter.mongodb;
 
-import org.eigenbase.rel.RelNode;
-import org.eigenbase.relopt.Convention;
-import org.eigenbase.relopt.RelOptTable;
-import org.eigenbase.util.Pair;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.util.Pair;
 
 import java.util.ArrayList;
 import java.util.List;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoRules.java
----------------------------------------------------------------------
diff --git a/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoRules.java b/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoRules.java
index 1b4b29e..88a45db 100644
--- a/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoRules.java
+++ b/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoRules.java
@@ -14,25 +14,39 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl.mongodb;
-
-import net.hydromatic.optiq.impl.java.JavaTypeFactory;
-import net.hydromatic.optiq.rules.java.RexImpTable;
-import net.hydromatic.optiq.rules.java.RexToLixTranslator;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.rel.convert.ConverterRule;
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.rex.*;
-import org.eigenbase.sql.SqlKind;
-import org.eigenbase.sql.fun.SqlStdOperatorTable;
-import org.eigenbase.sql.type.SqlTypeName;
-import org.eigenbase.sql.validate.SqlValidatorUtil;
-import org.eigenbase.trace.EigenbaseTrace;
-import org.eigenbase.util.Bug;
-
-import java.util.*;
+package org.apache.calcite.adapter.mongodb;
+
+import org.apache.calcite.adapter.enumerable.RexImpTable;
+import org.apache.calcite.adapter.enumerable.RexToLixTranslator;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelTrait;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.InvalidRelException;
+import org.apache.calcite.rel.RelCollationImpl;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.logical.LogicalAggregate;
+import org.apache.calcite.rel.logical.LogicalFilter;
+import org.apache.calcite.rel.logical.LogicalProject;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.validate.SqlValidatorUtil;
+import org.apache.calcite.util.Bug;
+import org.apache.calcite.util.trace.CalciteTrace;
+
+import java.util.AbstractList;
+import java.util.ArrayList;
+import java.util.List;
 import java.util.logging.Logger;
 
 /**
@@ -43,7 +57,7 @@ import java.util.logging.Logger;
 public class MongoRules {
   private MongoRules() {}
 
-  protected static final Logger LOGGER = EigenbaseTrace.getPlannerTracer();
+  protected static final Logger LOGGER = CalciteTrace.getPlannerTracer();
 
   public static final RelOptRule[] RULES = {
     MongoSortRule.INSTANCE,
@@ -185,90 +199,90 @@ public class MongoRules {
   }
 
   /**
-   * Rule to convert a {@link org.eigenbase.rel.SortRel} to a
-   * {@link MongoSortRel}.
+   * Rule to convert a {@link org.apache.calcite.rel.core.Sort} to a
+   * {@link MongoSort}.
    */
   private static class MongoSortRule extends MongoConverterRule {
     public static final MongoSortRule INSTANCE = new MongoSortRule();
 
     private MongoSortRule() {
-      super(SortRel.class, Convention.NONE, MongoRel.CONVENTION,
+      super(Sort.class, Convention.NONE, MongoRel.CONVENTION,
           "MongoSortRule");
     }
 
     public RelNode convert(RelNode rel) {
-      final SortRel sort = (SortRel) rel;
+      final Sort sort = (Sort) rel;
       final RelTraitSet traitSet =
           sort.getTraitSet().replace(out)
               .replace(sort.getCollation());
-      return new MongoSortRel(rel.getCluster(), traitSet,
-          convert(sort.getChild(), traitSet.replace(RelCollationImpl.EMPTY)),
+      return new MongoSort(rel.getCluster(), traitSet,
+          convert(sort.getInput(), traitSet.replace(RelCollationImpl.EMPTY)),
           sort.getCollation(), sort.offset, sort.fetch);
     }
   }
 
   /**
-   * Rule to convert a {@link org.eigenbase.rel.FilterRel} to a
-   * {@link MongoFilterRel}.
+   * Rule to convert a {@link org.apache.calcite.rel.logical.LogicalFilter} to a
+   * {@link MongoFilter}.
    */
   private static class MongoFilterRule extends MongoConverterRule {
     private static final MongoFilterRule INSTANCE = new MongoFilterRule();
 
     private MongoFilterRule() {
-      super(FilterRel.class, Convention.NONE, MongoRel.CONVENTION,
+      super(LogicalFilter.class, Convention.NONE, MongoRel.CONVENTION,
           "MongoFilterRule");
     }
 
     public RelNode convert(RelNode rel) {
-      final FilterRel filter = (FilterRel) rel;
+      final LogicalFilter filter = (LogicalFilter) rel;
       final RelTraitSet traitSet = filter.getTraitSet().replace(out);
-      return new MongoFilterRel(
+      return new MongoFilter(
           rel.getCluster(),
           traitSet,
-          convert(filter.getChild(), traitSet),
+          convert(filter.getInput(), traitSet),
           filter.getCondition());
     }
   }
 
   /**
-   * Rule to convert a {@link org.eigenbase.rel.ProjectRel} to a
-   * {@link MongoProjectRel}.
+   * Rule to convert a {@link org.apache.calcite.rel.logical.LogicalProject}
+   * to a {@link MongoProject}.
    */
   private static class MongoProjectRule extends MongoConverterRule {
     private static final MongoProjectRule INSTANCE = new MongoProjectRule();
 
     private MongoProjectRule() {
-      super(ProjectRel.class, Convention.NONE, MongoRel.CONVENTION,
+      super(LogicalProject.class, Convention.NONE, MongoRel.CONVENTION,
           "MongoProjectRule");
     }
 
     public RelNode convert(RelNode rel) {
-      final ProjectRel project = (ProjectRel) rel;
+      final LogicalProject project = (LogicalProject) rel;
       final RelTraitSet traitSet = project.getTraitSet().replace(out);
-      return new MongoProjectRel(project.getCluster(), traitSet,
-          convert(project.getChild(), traitSet), project.getProjects(),
-          project.getRowType(), ProjectRel.Flags.BOXED);
+      return new MongoProject(project.getCluster(), traitSet,
+          convert(project.getInput(), traitSet), project.getProjects(),
+          project.getRowType(), LogicalProject.Flags.BOXED);
     }
   }
 
 /*
 
   /**
-   * Rule to convert a {@link CalcRel} to an
+   * Rule to convert a {@link LogicalCalc} to an
    * {@link MongoCalcRel}.
    o/
   private static class MongoCalcRule
       extends MongoConverterRule {
     private MongoCalcRule(MongoConvention out) {
       super(
-          CalcRel.class,
+          LogicalCalc.class,
           Convention.NONE,
           out,
           "MongoCalcRule");
     }
 
     public RelNode convert(RelNode rel) {
-      final CalcRel calc = (CalcRel) rel;
+      final LogicalCalc calc = (LogicalCalc) rel;
 
       // If there's a multiset, let FarragoMultisetSplitter work on it
       // first.
@@ -283,7 +297,7 @@ public class MongoRules {
               calc.getChild(),
               calc.getTraitSet().replace(out)),
           calc.getProgram(),
-          ProjectRelBase.Flags.Boxed);
+          Project.Flags.Boxed);
     }
   }
 
@@ -291,7 +305,7 @@ public class MongoRules {
     private final RexProgram program;
 
     /**
-     * Values defined in {@link org.eigenbase.rel.ProjectRelBase.Flags}.
+     * Values defined in {@link org.apache.calcite.rel.core.Project.Flags}.
      o/
     protected int flags;
 
@@ -313,7 +327,7 @@ public class MongoRules {
     }
 
     public double getRows() {
-      return FilterRel.estimateFilteredRows(
+      return LogicalFilter.estimateFilteredRows(
           getChild(), program);
     }
 
@@ -421,26 +435,26 @@ public class MongoRules {
 */
 
   /**
-   * Rule to convert an {@link org.eigenbase.rel.AggregateRel} to an
-   * {@link MongoAggregateRel}.
+   * Rule to convert an {@link org.apache.calcite.rel.logical.LogicalAggregate}
+   * to an {@link MongoAggregate}.
    */
   private static class MongoAggregateRule extends MongoConverterRule {
     public static final RelOptRule INSTANCE = new MongoAggregateRule();
 
     private MongoAggregateRule() {
-      super(AggregateRel.class, Convention.NONE, MongoRel.CONVENTION,
+      super(LogicalAggregate.class, Convention.NONE, MongoRel.CONVENTION,
           "MongoAggregateRule");
     }
 
     public RelNode convert(RelNode rel) {
-      final AggregateRel agg = (AggregateRel) rel;
+      final LogicalAggregate agg = (LogicalAggregate) rel;
       final RelTraitSet traitSet =
           agg.getTraitSet().replace(out);
       try {
-        return new MongoAggregateRel(
+        return new MongoAggregate(
             rel.getCluster(),
             traitSet,
-            convert(agg.getChild(), traitSet),
+            convert(agg.getInput(), traitSet),
             agg.getGroupSet(),
             agg.getAggCallList());
       } catch (InvalidRelException e) {
@@ -452,21 +466,21 @@ public class MongoRules {
 
 /*
   /**
-   * Rule to convert an {@link org.eigenbase.rel.UnionRel} to a
+   * Rule to convert an {@link org.apache.calcite.rel.logical.Union} to a
    * {@link MongoUnionRel}.
    o/
   private static class MongoUnionRule
       extends MongoConverterRule {
     private MongoUnionRule(MongoConvention out) {
       super(
-          UnionRel.class,
+          Union.class,
           Convention.NONE,
           out,
           "MongoUnionRule");
     }
 
     public RelNode convert(RelNode rel) {
-      final UnionRel union = (UnionRel) rel;
+      final Union union = (Union) rel;
       final RelTraitSet traitSet =
           union.getTraitSet().replace(out);
       return new MongoUnionRel(
@@ -478,7 +492,7 @@ public class MongoRules {
   }
 
   public static class MongoUnionRel
-      extends UnionRelBase
+      extends Union
       implements MongoRel {
     public MongoUnionRel(
         RelOptCluster cluster,
@@ -493,8 +507,7 @@ public class MongoRules {
       return new MongoUnionRel(getCluster(), traitSet, inputs, all);
     }
 
-    @Override
-    public RelOptCost computeSelfCost(RelOptPlanner planner) {
+    @Override public RelOptCost computeSelfCost(RelOptPlanner planner) {
       return super.computeSelfCost(planner).multiplyBy(.1);
     }
 
@@ -504,7 +517,7 @@ public class MongoRules {
   }
 
   private static SqlString setOpSql(
-      SetOpRel setOpRel, MongoImplementor implementor, String op) {
+      SetOp setOpRel, MongoImplementor implementor, String op) {
     final SqlBuilder buf = new SqlBuilder(implementor.dialect);
     for (Ord<RelNode> input : Ord.zip(setOpRel.getInputs())) {
       if (input.i > 0) {
@@ -518,21 +531,21 @@ public class MongoRules {
   }
 
   /**
-   * Rule to convert an {@link org.eigenbase.rel.IntersectRel} to an
-   * {@link MongoIntersectRel}.
+   * Rule to convert an {@link org.apache.calcite.rel.logical.LogicalIntersect}
+   * to an {@link MongoIntersectRel}.
    o/
   private static class MongoIntersectRule
       extends MongoConverterRule {
     private MongoIntersectRule(MongoConvention out) {
       super(
-          IntersectRel.class,
+          LogicalIntersect.class,
           Convention.NONE,
           out,
           "MongoIntersectRule");
     }
 
     public RelNode convert(RelNode rel) {
-      final IntersectRel intersect = (IntersectRel) rel;
+      final LogicalIntersect intersect = (LogicalIntersect) rel;
       if (intersect.all) {
         return null; // INTERSECT ALL not implemented
       }
@@ -547,7 +560,7 @@ public class MongoRules {
   }
 
   public static class MongoIntersectRel
-      extends IntersectRelBase
+      extends Intersect
       implements MongoRel {
     public MongoIntersectRel(
         RelOptCluster cluster,
@@ -569,21 +582,21 @@ public class MongoRules {
   }
 
   /**
-   * Rule to convert an {@link org.eigenbase.rel.MinusRel} to an
-   * {@link MongoMinusRel}.
+   * Rule to convert an {@link org.apache.calcite.rel.logical.LogicalMinus}
+   * to an {@link MongoMinusRel}.
    o/
   private static class MongoMinusRule
       extends MongoConverterRule {
     private MongoMinusRule(MongoConvention out) {
       super(
-          MinusRel.class,
+          LogicalMinus.class,
           Convention.NONE,
           out,
           "MongoMinusRule");
     }
 
     public RelNode convert(RelNode rel) {
-      final MinusRel minus = (MinusRel) rel;
+      final LogicalMinus minus = (LogicalMinus) rel;
       if (minus.all) {
         return null; // EXCEPT ALL not implemented
       }
@@ -598,7 +611,7 @@ public class MongoRules {
   }
 
   public static class MongoMinusRel
-      extends MinusRelBase
+      extends Minus
       implements MongoRel {
     public MongoMinusRel(
         RelOptCluster cluster,
@@ -622,15 +635,14 @@ public class MongoRules {
   public static class MongoValuesRule extends MongoConverterRule {
     private MongoValuesRule(MongoConvention out) {
       super(
-          ValuesRel.class,
+          LogicalValues.class,
           Convention.NONE,
           out,
           "MongoValuesRule");
     }
 
-    @Override
-    public RelNode convert(RelNode rel) {
-      ValuesRel valuesRel = (ValuesRel) rel;
+    @Override public RelNode convert(RelNode rel) {
+      LogicalValues valuesRel = (LogicalValues) rel;
       return new MongoValuesRel(
           valuesRel.getCluster(),
           valuesRel.getRowType(),
@@ -640,7 +652,7 @@ public class MongoRules {
   }
 
   public static class MongoValuesRel
-      extends ValuesRelBase
+      extends Values
       implements MongoRel {
     MongoValuesRel(
         RelOptCluster cluster,
@@ -650,8 +662,7 @@ public class MongoRules {
       super(cluster, rowType, tuples, traitSet);
     }
 
-    @Override
-    public RelNode copy(
+    @Override public RelNode copy(
         RelTraitSet traitSet, List<RelNode> inputs) {
       assert inputs.isEmpty();
       return new MongoValuesRel(

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoSchema.java
----------------------------------------------------------------------
diff --git a/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoSchema.java b/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoSchema.java
index 34905ff..5bfc3f6 100644
--- a/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoSchema.java
+++ b/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoSchema.java
@@ -14,17 +14,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl.mongodb;
+package org.apache.calcite.adapter.mongodb;
 
-import net.hydromatic.optiq.*;
-import net.hydromatic.optiq.impl.AbstractSchema;
+import org.apache.calcite.schema.Table;
+import org.apache.calcite.schema.impl.AbstractSchema;
 
 import com.google.common.collect.ImmutableMap;
-
 import com.mongodb.DB;
 import com.mongodb.MongoClient;
 
-import java.util.*;
+import java.util.Map;
 
 /**
  * Schema mapped onto a directory of MONGO files. Each table in the schema
@@ -49,8 +48,7 @@ public class MongoSchema extends AbstractSchema {
     }
   }
 
-  @Override
-  protected Map<String, Table> getTableMap() {
+  @Override protected Map<String, Table> getTableMap() {
     final ImmutableMap.Builder<String, Table> builder = ImmutableMap.builder();
     for (String collectionName : mongoDb.getCollectionNames()) {
       builder.put(collectionName, new MongoTable(collectionName));

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoSchemaFactory.java
----------------------------------------------------------------------
diff --git a/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoSchemaFactory.java b/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoSchemaFactory.java
index 10c57fd..46ceddb 100644
--- a/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoSchemaFactory.java
+++ b/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoSchemaFactory.java
@@ -14,9 +14,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl.mongodb;
+package org.apache.calcite.adapter.mongodb;
 
-import net.hydromatic.optiq.*;
+import org.apache.calcite.schema.Schema;
+import org.apache.calcite.schema.SchemaFactory;
+import org.apache.calcite.schema.SchemaPlus;
 
 import java.util.Map;
 


[20/58] [abbrv] [partial] incubator-calcite git commit: [CALCITE-306] Standardize code style for "import package.*; "

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/ChunkList.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/ChunkList.java b/core/src/main/java/org/apache/calcite/util/ChunkList.java
index a8a4a0c..66644e9 100644
--- a/core/src/main/java/org/apache/calcite/util/ChunkList.java
+++ b/core/src/main/java/org/apache/calcite/util/ChunkList.java
@@ -14,9 +14,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.util;
+package org.apache.calcite.util;
 
-import java.util.*;
+import java.util.AbstractSequentialList;
+import java.util.Collection;
+import java.util.LinkedList;
+import java.util.ListIterator;
+import java.util.NoSuchElementException;
 
 /**
  * Implementation of list similar to {@link LinkedList}, but stores elements
@@ -24,6 +28,8 @@ import java.util.*;
  *
  * <p>ArrayList has O(n) insertion and deletion into the middle of the list.
  * ChunkList insertion and deletion are O(1).</p>
+ *
+ * @param <E> element type
  */
 public class ChunkList<E> extends AbstractSequentialList<E> {
   private static final int HEADER_SIZE = 3;
@@ -91,18 +97,15 @@ public class ChunkList<E> extends AbstractSequentialList<E> {
     return true;
   }
 
-  @Override
-  public ListIterator<E> listIterator(int index) {
+  @Override public ListIterator<E> listIterator(int index) {
     return locate(index);
   }
 
-  @Override
-  public int size() {
+  @Override public int size() {
     return size;
   }
 
-  @Override
-  public boolean add(E element) {
+  @Override public boolean add(E element) {
     Object[] chunk = last;
     int occupied;
     if (chunk == null) {
@@ -124,8 +127,7 @@ public class ChunkList<E> extends AbstractSequentialList<E> {
     return true;
   }
 
-  @Override
-  public void add(int index, E element) {
+  @Override public void add(int index, E element) {
     if (index == size) {
       add(element);
     } else {
@@ -185,6 +187,7 @@ public class ChunkList<E> extends AbstractSequentialList<E> {
     }
   }
 
+  /** Iterator over a {@link ChunkList}. */
   private class ChunkListIterator implements ListIterator<E> {
     private Object[] chunk;
     private int startIndex;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/ClosableAllocation.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/ClosableAllocation.java b/core/src/main/java/org/apache/calcite/util/ClosableAllocation.java
index 75f6334..9568772 100644
--- a/core/src/main/java/org/apache/calcite/util/ClosableAllocation.java
+++ b/core/src/main/java/org/apache/calcite/util/ClosableAllocation.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.util;
+package org.apache.calcite.util;
 
 /**
  * ClosableAllocation represents an object which requires a call in order to

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/ClosableAllocationOwner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/ClosableAllocationOwner.java b/core/src/main/java/org/apache/calcite/util/ClosableAllocationOwner.java
index 8996306..3915657 100644
--- a/core/src/main/java/org/apache/calcite/util/ClosableAllocationOwner.java
+++ b/core/src/main/java/org/apache/calcite/util/ClosableAllocationOwner.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.util;
+package org.apache.calcite.util;
 
 /**
  * ClosableAllocationOwner represents an object which can take ownership of

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/Compatible.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/Compatible.java b/core/src/main/java/org/apache/calcite/util/Compatible.java
index 0c8d09d..41b5123 100644
--- a/core/src/main/java/org/apache/calcite/util/Compatible.java
+++ b/core/src/main/java/org/apache/calcite/util/Compatible.java
@@ -14,13 +14,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.util;
+package org.apache.calcite.util;
 
 import com.google.common.base.Function;
-import com.google.common.collect.*;
+import com.google.common.collect.ImmutableSortedMap;
+import com.google.common.collect.ImmutableSortedSet;
+import com.google.common.collect.Maps;
 
-import java.lang.reflect.*;
-import java.util.*;
+import java.lang.reflect.InvocationHandler;
+import java.lang.reflect.Method;
+import java.lang.reflect.Proxy;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.NavigableSet;
+import java.util.Set;
 
 /** Compatibility layer.
  *

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/CompatibleGuava11.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/CompatibleGuava11.java b/core/src/main/java/org/apache/calcite/util/CompatibleGuava11.java
index 791558e..8c2a916 100644
--- a/core/src/main/java/org/apache/calcite/util/CompatibleGuava11.java
+++ b/core/src/main/java/org/apache/calcite/util/CompatibleGuava11.java
@@ -14,13 +14,29 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.util;
+package org.apache.calcite.util;
 
-import com.google.common.base.*;
+import com.google.common.base.Function;
 import com.google.common.base.Objects;
-import com.google.common.collect.*;
-
-import java.util.*;
+import com.google.common.collect.Collections2;
+import com.google.common.collect.ForwardingSet;
+import com.google.common.collect.ImmutableSortedMap;
+import com.google.common.collect.ImmutableSortedSet;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Multiset;
+import com.google.common.collect.Sets;
+
+import java.util.AbstractCollection;
+import java.util.AbstractMap;
+import java.util.AbstractSet;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.NavigableSet;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.TreeSet;
 
 import static com.google.common.base.Preconditions.checkNotNull;
 
@@ -41,13 +57,11 @@ class CompatibleGuava11 {
    * {@code removeAll} implementation.
    */
   abstract static class ImprovedAbstractSet<E> extends AbstractSet<E> {
-    @Override
-    public boolean removeAll(Collection<?> c) {
+    @Override public boolean removeAll(Collection<?> c) {
       return removeAllImpl(this, c);
     }
 
-    @Override
-    public boolean retainAll(Collection<?> c) {
+    @Override public boolean retainAll(Collection<?> c) {
       return super.retainAll(checkNotNull(c)); // GWT compatibility
     }
   }
@@ -140,8 +154,7 @@ class CompatibleGuava11 {
   static <K, V> Iterator<K> keyIterator(
       Iterator<Map.Entry<K, V>> entryIterator) {
     return new TransformedIterator<Map.Entry<K, V>, K>(entryIterator) {
-      @Override
-      K transform(Map.Entry<K, V> entry) {
+      @Override K transform(Map.Entry<K, V> entry) {
         return entry.getKey();
       }
     };
@@ -182,18 +195,15 @@ class CompatibleGuava11 {
 
   private static <E> Set<E> removeOnlySet(final Set<E> set) {
     return new ForwardingSet<E>() {
-      @Override
-      protected Set<E> delegate() {
+      @Override protected Set<E> delegate() {
         return set;
       }
 
-      @Override
-      public boolean add(E element) {
+      @Override public boolean add(E element) {
         throw new UnsupportedOperationException();
       }
 
-      @Override
-      public boolean addAll(Collection<? extends E> es) {
+      @Override public boolean addAll(Collection<? extends E> es) {
         throw new UnsupportedOperationException();
       }
     };
@@ -202,8 +212,7 @@ class CompatibleGuava11 {
   private static <K, V> Iterator<Map.Entry<K, V>> asSetEntryIterator(
       Set<K> set, final Function<? super K, V> function) {
     return new TransformedIterator<K, Map.Entry<K, V>>(set.iterator()) {
-      @Override
-      Map.Entry<K, V> transform(K key) {
+      @Override Map.Entry<K, V> transform(K key) {
         return Maps.immutableEntry(key, function.apply(key));
       }
     };
@@ -223,30 +232,25 @@ class CompatibleGuava11 {
       this.function = checkNotNull(function);
     }
 
-    @Override
-    public Set<K> keySet() {
+    @Override public Set<K> keySet() {
       // probably not worth caching
       return removeOnlySet(backingSet());
     }
 
-    @Override
-    public Collection<V> values() {
+    @Override public Collection<V> values() {
       // probably not worth caching
       return Collections2.transform(set, function);
     }
 
-    @Override
-    public int size() {
+    @Override public int size() {
       return backingSet().size();
     }
 
-    @Override
-    public boolean containsKey(Object key) {
+    @Override public boolean containsKey(Object key) {
       return backingSet().contains(key);
     }
 
-    @Override
-    public V get(Object key) {
+    @Override public V get(Object key) {
       if (backingSet().contains(key)) {
         @SuppressWarnings("unchecked") // unsafe, but Javadoc warns about it
             K k = (K) key;
@@ -256,8 +260,7 @@ class CompatibleGuava11 {
       }
     }
 
-    @Override
-    public V remove(Object key) {
+    @Override public V remove(Object key) {
       if (backingSet().remove(key)) {
         @SuppressWarnings("unchecked") // unsafe, but Javadoc warns about it
             K k = (K) key;
@@ -267,21 +270,17 @@ class CompatibleGuava11 {
       }
     }
 
-    @Override
-    public void clear() {
+    @Override public void clear() {
       backingSet().clear();
     }
 
-    @Override
-    protected Set<Map.Entry<K, V>> createEntrySet() {
+    @Override protected Set<Map.Entry<K, V>> createEntrySet() {
       return new EntrySet<K, V>() {
-        @Override
-        Map<K, V> map() {
+        @Override Map<K, V> map() {
           return AsMapView.this;
         }
 
-        @Override
-        public Iterator<Map.Entry<K, V>> iterator() {
+        @Override public Iterator<Map.Entry<K, V>> iterator() {
           return asSetEntryIterator(backingSet(), function);
         }
       };
@@ -357,8 +356,7 @@ class CompatibleGuava11 {
   static <K, V> Iterator<V> valueIterator(
       Iterator<Map.Entry<K, V>> entryIterator) {
     return new TransformedIterator<Map.Entry<K, V>, V>(entryIterator) {
-      @Override
-      V transform(Map.Entry<K, V> entry) {
+      @Override V transform(Map.Entry<K, V> entry) {
         return entry.getValue();
       }
     };

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/CompositeList.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/CompositeList.java b/core/src/main/java/org/apache/calcite/util/CompositeList.java
index 00fc24f..85ed62c 100644
--- a/core/src/main/java/org/apache/calcite/util/CompositeList.java
+++ b/core/src/main/java/org/apache/calcite/util/CompositeList.java
@@ -14,12 +14,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.util;
-
-import java.util.*;
+package org.apache.calcite.util;
 
 import com.google.common.collect.ImmutableList;
 
+import java.util.AbstractList;
+import java.util.List;
+
 /**
  * Read-only list that is the concatenation of sub-lists.
  *

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/CompositeMap.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/CompositeMap.java b/core/src/main/java/org/apache/calcite/util/CompositeMap.java
index 748cb1d..66a3e40 100644
--- a/core/src/main/java/org/apache/calcite/util/CompositeMap.java
+++ b/core/src/main/java/org/apache/calcite/util/CompositeMap.java
@@ -14,12 +14,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.util;
+package org.apache.calcite.util;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 
-import java.util.*;
+import java.util.Collection;
+import java.util.LinkedHashSet;
+import java.util.Map;
+import java.util.Set;
 
 /** Unmodifiable view onto multiple backing maps. An element occurs in the map
  * if it occurs in any of the backing maps; the value is the value that occurs

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/CompoundClosableAllocation.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/CompoundClosableAllocation.java b/core/src/main/java/org/apache/calcite/util/CompoundClosableAllocation.java
index 04c2239..283f064 100644
--- a/core/src/main/java/org/apache/calcite/util/CompoundClosableAllocation.java
+++ b/core/src/main/java/org/apache/calcite/util/CompoundClosableAllocation.java
@@ -14,9 +14,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.util;
+package org.apache.calcite.util;
 
-import java.util.*;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.ListIterator;
 
 /**
  * CompoundClosableAllocation represents a collection of ClosableAllocations

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/ControlFlowException.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/ControlFlowException.java b/core/src/main/java/org/apache/calcite/util/ControlFlowException.java
index f656157..0740b38 100644
--- a/core/src/main/java/org/apache/calcite/util/ControlFlowException.java
+++ b/core/src/main/java/org/apache/calcite/util/ControlFlowException.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.util;
+package org.apache.calcite.util;
 
 /**
  * Exception intended to be used for control flow, as opposed to the usual

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/ConversionUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/ConversionUtil.java b/core/src/main/java/org/apache/calcite/util/ConversionUtil.java
index a577b14..193cc7c 100644
--- a/core/src/main/java/org/apache/calcite/util/ConversionUtil.java
+++ b/core/src/main/java/org/apache/calcite/util/ConversionUtil.java
@@ -14,12 +14,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.util14;
+package org.apache.calcite.util;
 
-import java.nio.*;
-import java.text.*;
+import java.nio.ByteOrder;
+import java.text.NumberFormat;
 
-import static org.eigenbase.util.Static.RESOURCE;
+import static org.apache.calcite.util.Static.RESOURCE;
 
 /**
  * Utility functions for converting from one type to another
@@ -79,8 +79,8 @@ public class ConversionUtil {
   }
 
   /**
-   * Converts a string into a byte array. The inverse of {@link
-   * #toStringFromByteArray(byte[], int)}.
+   * Converts a string into a byte array. The inverse of
+   * {@link #toStringFromByteArray(byte[], int)}.
    */
   public static byte[] toByteArrayFromString(
       String value,

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/DateTimeUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/DateTimeUtil.java b/core/src/main/java/org/apache/calcite/util/DateTimeUtil.java
index 267d59d..82b8665 100644
--- a/core/src/main/java/org/apache/calcite/util/DateTimeUtil.java
+++ b/core/src/main/java/org/apache/calcite/util/DateTimeUtil.java
@@ -14,11 +14,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.util14;
+package org.apache.calcite.util;
 
-import java.text.*;
-
-import java.util.*;
+import java.text.NumberFormat;
+import java.text.ParsePosition;
+import java.text.SimpleDateFormat;
+import java.util.Calendar;
+import java.util.TimeZone;
 
 /**
  * Utility functions for datetime types: date, time, timestamp. Refactored from

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/DelegatingInvocationHandler.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/DelegatingInvocationHandler.java b/core/src/main/java/org/apache/calcite/util/DelegatingInvocationHandler.java
index 7f5293a..8df789f 100644
--- a/core/src/main/java/org/apache/calcite/util/DelegatingInvocationHandler.java
+++ b/core/src/main/java/org/apache/calcite/util/DelegatingInvocationHandler.java
@@ -14,9 +14,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.util;
+package org.apache.calcite.util;
 
-import java.lang.reflect.*;
+import java.lang.reflect.InvocationHandler;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
 
 /**
  * A class derived from <code>DelegatingInvocationHandler</code> handles a

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/Filterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/Filterator.java b/core/src/main/java/org/apache/calcite/util/Filterator.java
index aab8a49..6b282bd 100644
--- a/core/src/main/java/org/apache/calcite/util/Filterator.java
+++ b/core/src/main/java/org/apache/calcite/util/Filterator.java
@@ -14,16 +14,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.util;
+package org.apache.calcite.util;
 
-import java.util.*;
+import java.util.Iterator;
+import java.util.NoSuchElementException;
 
 /**
  * Filtered iterator class: an iterator that includes only elements that are
- * instanceof a specified class. Apologies for the dorky name.
+ * instanceof a specified class.
  *
- * @see Util#cast(List, Class)
+ * <p>Apologies for the dorky name.
+ *
+ * @see Util#cast(java.util.List, Class)
  * @see Util#cast(Iterator, Class)
+ *
+ * @param <E> Element type
  */
 public class Filterator<E> implements Iterator<E> {
   //~ Instance fields --------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/Glossary.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/Glossary.java b/core/src/main/java/org/apache/calcite/util/Glossary.java
index f4edc79..be32145 100644
--- a/core/src/main/java/org/apache/calcite/util/Glossary.java
+++ b/core/src/main/java/org/apache/calcite/util/Glossary.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.util;
+package org.apache.calcite.util;
 
 /**
  * A collection of terms.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/Holder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/Holder.java b/core/src/main/java/org/apache/calcite/util/Holder.java
index 090697c..938fb4d 100644
--- a/core/src/main/java/org/apache/calcite/util/Holder.java
+++ b/core/src/main/java/org/apache/calcite/util/Holder.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.util;
+package org.apache.calcite.util;
 
 /**
  * A mutable slot that can contain one object.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/ImmutableIntList.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/ImmutableIntList.java b/core/src/main/java/org/apache/calcite/util/ImmutableIntList.java
index 1e1c2e3..ace0c95 100644
--- a/core/src/main/java/org/apache/calcite/util/ImmutableIntList.java
+++ b/core/src/main/java/org/apache/calcite/util/ImmutableIntList.java
@@ -14,16 +14,23 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.util;
+package org.apache.calcite.util;
 
-import java.lang.reflect.Array;
-import java.util.*;
-
-import net.hydromatic.optiq.runtime.FlatLists;
+import org.apache.calcite.runtime.FlatLists;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.UnmodifiableListIterator;
 
+import java.lang.reflect.Array;
+import java.util.AbstractList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.NoSuchElementException;
+
 /**
  * An immutable list of {@link Integer} values backed by an array of
  * {@code int}s.
@@ -83,13 +90,11 @@ public class ImmutableIntList extends FlatLists.AbstractFlatList<Integer> {
     return new ImmutableIntList(ints);
   }
 
-  @Override
-  public int hashCode() {
+  @Override public int hashCode() {
     return Arrays.hashCode(ints);
   }
 
-  @Override
-  public boolean equals(Object obj) {
+  @Override public boolean equals(Object obj) {
     return this == obj
         || obj instanceof ImmutableIntList
         && Arrays.equals(ints, ((ImmutableIntList) obj).ints)
@@ -97,13 +102,11 @@ public class ImmutableIntList extends FlatLists.AbstractFlatList<Integer> {
         && obj.equals(this);
   }
 
-  @Override
-  public String toString() {
+  @Override public String toString() {
     return Arrays.toString(ints);
   }
 
-  @Override
-  public boolean isEmpty() {
+  @Override public boolean isEmpty() {
     return ints.length == 0;
   }
 
@@ -218,27 +221,25 @@ public class ImmutableIntList extends FlatLists.AbstractFlatList<Integer> {
     };
   }
 
+  /** Special sub-class of {@link ImmutableIntList} that is always
+   * empty and has only one instance. */
   private static class EmptyImmutableIntList extends ImmutableIntList {
-    @Override
-    public Object[] toArray() {
+    @Override public Object[] toArray() {
       return EMPTY_ARRAY;
     }
 
-    @Override
-    public <T> T[] toArray(T[] a) {
+    @Override public <T> T[] toArray(T[] a) {
       if (a.length > 0) {
         a[0] = null;
       }
       return a;
     }
 
-    @Override
-    public Iterator<Integer> iterator() {
+    @Override public Iterator<Integer> iterator() {
       return Collections.<Integer>emptyList().iterator();
     }
 
-    @Override
-    public ListIterator<Integer> listIterator() {
+    @Override public ListIterator<Integer> listIterator() {
       return Collections.<Integer>emptyList().listIterator();
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/ImmutableNullableList.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/ImmutableNullableList.java b/core/src/main/java/org/apache/calcite/util/ImmutableNullableList.java
index a3367aa..1e92f7b 100644
--- a/core/src/main/java/org/apache/calcite/util/ImmutableNullableList.java
+++ b/core/src/main/java/org/apache/calcite/util/ImmutableNullableList.java
@@ -14,12 +14,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.util;
-
-import java.util.*;
+package org.apache.calcite.util;
 
 import com.google.common.collect.ImmutableList;
 
+import java.util.AbstractList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+
 /**
  * An immutable list that may contain null values.
  *

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/IntList.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/IntList.java b/core/src/main/java/org/apache/calcite/util/IntList.java
index 33269ea..7edb508 100644
--- a/core/src/main/java/org/apache/calcite/util/IntList.java
+++ b/core/src/main/java/org/apache/calcite/util/IntList.java
@@ -14,9 +14,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.util;
+package org.apache.calcite.util;
 
-import java.util.*;
+import java.util.AbstractList;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
 
 /**
  * Extension to {@link ArrayList} to help build an array of <code>int</code>

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/IntegerIntervalSet.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/IntegerIntervalSet.java b/core/src/main/java/org/apache/calcite/util/IntegerIntervalSet.java
index 5fb726d..1105461 100644
--- a/core/src/main/java/org/apache/calcite/util/IntegerIntervalSet.java
+++ b/core/src/main/java/org/apache/calcite/util/IntegerIntervalSet.java
@@ -14,15 +14,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.util;
+package org.apache.calcite.util;
+
+import org.apache.calcite.linq4j.Enumerator;
+import org.apache.calcite.linq4j.Linq4j;
 
 import java.util.AbstractSet;
 import java.util.Iterator;
 import java.util.Set;
 
-import net.hydromatic.linq4j.Enumerator;
-import net.hydromatic.linq4j.Linq4j;
-
 /**
  * A set of non-negative integers defined by a sequence of points, intervals,
  * and exclusions.
@@ -75,13 +75,11 @@ public class IntegerIntervalSet extends AbstractSet<Integer> {
   }
 
   @SuppressWarnings("NullableProblems")
-  @Override
-  public Iterator<Integer> iterator() {
+  @Override public Iterator<Integer> iterator() {
     return Linq4j.enumeratorIterator(enumerator());
   }
 
-  @Override
-  public int size() {
+  @Override public int size() {
     int n = 0;
     Enumerator<Integer> e = enumerator();
     while (e.moveNext()) {
@@ -130,8 +128,7 @@ public class IntegerIntervalSet extends AbstractSet<Integer> {
     };
   }
 
-  @Override
-  public boolean contains(Object o) {
+  @Override public boolean contains(Object o) {
     return o instanceof Number
         && contains(((Number) o).intValue());
   }
@@ -150,6 +147,7 @@ public class IntegerIntervalSet extends AbstractSet<Integer> {
     return bs[0];
   }
 
+  /** A callback. */
   private interface Handler {
     void range(int start, int end, boolean exclude);
   }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/JsonBuilder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/JsonBuilder.java b/core/src/main/java/org/apache/calcite/util/JsonBuilder.java
index eb95a53..378fd63 100644
--- a/core/src/main/java/org/apache/calcite/util/JsonBuilder.java
+++ b/core/src/main/java/org/apache/calcite/util/JsonBuilder.java
@@ -14,15 +14,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.util;
+package org.apache.calcite.util;
+
+import org.apache.calcite.runtime.Spaces;
 
 import java.util.ArrayList;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 
-import net.hydromatic.optiq.runtime.Spaces;
-
 /**
  * Builder for JSON documents (represented as {@link List}, {@link Map},
  * {@link String}, {@link Boolean}, {@link Long}).

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/NlsString.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/NlsString.java b/core/src/main/java/org/apache/calcite/util/NlsString.java
index 087a0c7..1a49e5d 100644
--- a/core/src/main/java/org/apache/calcite/util/NlsString.java
+++ b/core/src/main/java/org/apache/calcite/util/NlsString.java
@@ -14,21 +14,27 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.util;
+package org.apache.calcite.util;
 
-import java.nio.*;
-import java.nio.charset.*;
-import java.util.List;
+import org.apache.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.SqlCollation;
+import org.apache.calcite.sql.SqlUtil;
 
-import org.eigenbase.sql.*;
+import com.google.common.base.Objects;
 
-import net.hydromatic.optiq.runtime.SqlFunctions;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.IllegalCharsetNameException;
+import java.nio.charset.UnsupportedCharsetException;
+import java.util.List;
 
-import static org.eigenbase.util.Static.RESOURCE;
+import static org.apache.calcite.util.Static.RESOURCE;
 
 /**
- * A string, optionally with {@link Charset character set} and {@link
- * SqlCollation}. It is immutable.
+ * A string, optionally with {@link Charset character set} and
+ * {@link SqlCollation}. It is immutable.
  */
 public class NlsString implements Comparable<NlsString> {
   //~ Instance fields --------------------------------------------------------
@@ -100,9 +106,9 @@ public class NlsString implements Comparable<NlsString> {
       return false;
     }
     NlsString that = (NlsString) obj;
-    return Util.equal(value, that.value)
-        && Util.equal(charsetName, that.charsetName)
-        && Util.equal(collation, that.collation);
+    return Objects.equal(value, that.value)
+        && Objects.equal(charsetName, that.charsetName)
+        && Objects.equal(collation, that.collation);
   }
 
   // implement Comparable

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/NumberUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/NumberUtil.java b/core/src/main/java/org/apache/calcite/util/NumberUtil.java
index 3231464..4d5f71a 100644
--- a/core/src/main/java/org/apache/calcite/util/NumberUtil.java
+++ b/core/src/main/java/org/apache/calcite/util/NumberUtil.java
@@ -14,11 +14,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.util14;
+package org.apache.calcite.util;
 
-import java.math.*;
-
-import java.text.*;
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.text.DecimalFormat;
+import java.text.NumberFormat;
 
 /**
  * Utility functions for working with numbers This class is JDK 1.4 compatible.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/Pair.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/Pair.java b/core/src/main/java/org/apache/calcite/util/Pair.java
index d5bb8b9..a01e9e7 100644
--- a/core/src/main/java/org/apache/calcite/util/Pair.java
+++ b/core/src/main/java/org/apache/calcite/util/Pair.java
@@ -14,19 +14,28 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.util;
-
-import java.io.Serializable;
-import java.util.*;
+package org.apache.calcite.util;
 
+import com.google.common.base.Objects;
 import com.google.common.collect.Iterators;
 
+import java.io.Serializable;
+import java.util.AbstractList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
 /**
  * Pair of objects.
  *
  * <p>Because a pair implements {@link #equals(Object)}, {@link #hashCode()} and
  * {@link #compareTo(Pair)}, it can be used in any kind of
  * {@link java.util.Collection}.
+ *
+ * @param <T1> Left-hand type
+ * @param <T2> Right-hand type
  */
 public class Pair<T1, T2>
     implements Comparable<Pair<T1, T2>>, Map.Entry<T1, T2>, Serializable {
@@ -70,8 +79,8 @@ public class Pair<T1, T2>
   public boolean equals(Object obj) {
     return this == obj
         || (obj instanceof Pair)
-        && Util.equal(this.left, ((Pair) obj).left)
-        && Util.equal(this.right, ((Pair) obj).right);
+        && Objects.equal(this.left, ((Pair) obj).left)
+        && com.google.common.base.Objects.equal(this.right, ((Pair) obj).right);
   }
 
   public int hashCode() {
@@ -147,14 +156,14 @@ public class Pair<T1, T2>
   }
 
   /**
-   * Converts two lists into a list of {@link org.eigenbase.util.Pair}s,
+   * Converts two lists into a list of {@link Pair}s,
    * whose length is the lesser of the lengths of the
    * source lists.
    *
    * @param ks Left list
    * @param vs Right list
    * @return List of pairs
-   * @see net.hydromatic.linq4j.Ord#zip(java.util.List)
+   * @see org.apache.calcite.linq4j.Ord#zip(java.util.List)
    */
   public static <K, V> List<Pair<K, V>> zip(List<K> ks, List<V> vs) {
     return zip(ks, vs, false);
@@ -170,7 +179,7 @@ public class Pair<T1, T2>
    * @param vs     Right list
    * @param strict Whether to fail if lists have different size
    * @return List of pairs
-   * @see net.hydromatic.linq4j.Ord#zip(java.util.List)
+   * @see org.apache.calcite.linq4j.Ord#zip(java.util.List)
    */
   public static <K, V> List<Pair<K, V>> zip(
       final List<K> ks,

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/PartiallyOrderedSet.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/PartiallyOrderedSet.java b/core/src/main/java/org/apache/calcite/util/PartiallyOrderedSet.java
index 33ef912..200e1c9 100644
--- a/core/src/main/java/org/apache/calcite/util/PartiallyOrderedSet.java
+++ b/core/src/main/java/org/apache/calcite/util/PartiallyOrderedSet.java
@@ -14,9 +14,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.util;
-
-import java.util.*;
+package org.apache.calcite.util;
+
+import java.util.AbstractList;
+import java.util.AbstractSet;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.Set;
 
 /**
  * Partially-ordered set.
@@ -98,8 +109,7 @@ public class PartiallyOrderedSet<E> extends AbstractSet<E> {
   }
 
   @SuppressWarnings("NullableProblems")
-  @Override
-  public Iterator<E> iterator() {
+  @Override public Iterator<E> iterator() {
     final Iterator<E> iterator = map.keySet().iterator();
     return new Iterator<E>() {
       E previous;
@@ -124,19 +134,16 @@ public class PartiallyOrderedSet<E> extends AbstractSet<E> {
     };
   }
 
-  @Override
-  public int size() {
+  @Override public int size() {
     return map.size();
   }
 
-  @Override
-  public boolean contains(Object o) {
+  @Override public boolean contains(Object o) {
     //noinspection SuspiciousMethodCalls
     return map.containsKey(o);
   }
 
-  @Override
-  public boolean remove(Object o) {
+  @Override public boolean remove(Object o) {
     @SuppressWarnings("SuspiciousMethodCalls")
     final Node<E> node = map.remove(o);
     if (node == null) {
@@ -168,8 +175,7 @@ public class PartiallyOrderedSet<E> extends AbstractSet<E> {
   /**
    * Adds an element to this lattice.
    */
-  @Override
-  public boolean add(E e) {
+  @Override public boolean add(E e) {
     assert e != null;
     assert !DEBUG || isValid(true);
     Node<E> node = map.get(e);
@@ -588,8 +594,7 @@ public class PartiallyOrderedSet<E> extends AbstractSet<E> {
     return new StripList<E>(bottomNode.parentList);
   }
 
-  @Override
-  public void clear() {
+  @Override public void clear() {
     map.clear();
     assert topNode.parentList.isEmpty();
     topNode.childList.clear();
@@ -673,8 +678,7 @@ public class PartiallyOrderedSet<E> extends AbstractSet<E> {
       this.e = e;
     }
 
-    @Override
-    public String toString() {
+    @Override public String toString() {
       return e.toString();
     }
   }
@@ -693,8 +697,7 @@ public class PartiallyOrderedSet<E> extends AbstractSet<E> {
       this.description = top ? "top" : "bottom";
     }
 
-    @Override
-    public String toString() {
+    @Override public String toString() {
       return description;
     }
   }
@@ -739,13 +742,11 @@ public class PartiallyOrderedSet<E> extends AbstractSet<E> {
       this.list = list;
     }
 
-    @Override
-    public E get(int index) {
+    @Override public E get(int index) {
       return list.get(index).e;
     }
 
-    @Override
-    public int size() {
+    @Override public int size() {
       return list.size();
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/Permutation.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/Permutation.java b/core/src/main/java/org/apache/calcite/util/Permutation.java
index f7a7471..4cbff86 100644
--- a/core/src/main/java/org/apache/calcite/util/Permutation.java
+++ b/core/src/main/java/org/apache/calcite/util/Permutation.java
@@ -14,11 +14,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.util;
+package org.apache.calcite.util;
 
-import java.util.*;
+import org.apache.calcite.util.mapping.IntPair;
+import org.apache.calcite.util.mapping.Mapping;
+import org.apache.calcite.util.mapping.MappingType;
+import org.apache.calcite.util.mapping.Mappings;
 
-import org.eigenbase.util.mapping.*;
+import java.util.Arrays;
+import java.util.Iterator;
 
 /**
  * Represents a mapping which reorders elements in an array.
@@ -109,8 +113,7 @@ public class Permutation implements Mapping, Mappings.TargetMapping {
 
   public void clear() {
     throw new UnsupportedOperationException(
-        "Cannot clear: permutation must always contain one mapping per "
-        + "element");
+        "Cannot clear: permutation must always contain one mapping per element");
   }
 
   /**
@@ -450,13 +453,15 @@ public class Permutation implements Mapping, Mappings.TargetMapping {
     for (int i = 0; i < size; i++) {
       int target = targets[i];
       if (sources[target] != i) {
-        assert !fail : "source[" + target + "] = " + sources[target]
+        assert !fail
+            : "source[" + target + "] = " + sources[target]
             + ", should be " + i;
         return false;
       }
       int source = sources[i];
       if (targets[source] != i) {
-        assert !fail : "target[" + source + "] = " + targets[source]
+        assert !fail
+            : "target[" + source + "] = " + targets[source]
             + ", should be " + i;
         return false;
       }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/ReflectUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/ReflectUtil.java b/core/src/main/java/org/apache/calcite/util/ReflectUtil.java
index 6695281..4a29253 100644
--- a/core/src/main/java/org/apache/calcite/util/ReflectUtil.java
+++ b/core/src/main/java/org/apache/calcite/util/ReflectUtil.java
@@ -14,16 +14,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.util;
-
-import java.lang.reflect.*;
-
-import java.nio.*;
-
-import java.util.*;
+package org.apache.calcite.util;
 
 import com.google.common.collect.ImmutableList;
 
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
 /**
  * Static utilities for Java reflection.
  */
@@ -181,11 +184,13 @@ public abstract class ReflectUtil {
   }
 
   /**
-   * Implements the {@link Glossary#VISITOR_PATTERN} via reflection. The basic
-   * technique is taken from <a
+   * Implements the {@link org.apache.calcite.util.Glossary#VISITOR_PATTERN} via
+   * reflection. The basic technique is taken from <a
    * href="http://www.javaworld.com/javaworld/javatips/jw-javatip98.html">a
-   * Javaworld article</a>. For an example of how to use it, see {@code
-   * ReflectVisitorTest}. Visit method lookup follows the same rules as if
+   * Javaworld article</a>. For an example of how to use it, see
+   * {@code ReflectVisitorTest}.
+   *
+   * <p>Visit method lookup follows the same rules as if
    * compile-time resolution for VisitorClass.visit(VisiteeClass) were
    * performed. An ambiguous match due to multiple interface inheritance
    * results in an IllegalArgumentException. A non-match is indicated by
@@ -393,9 +398,8 @@ public abstract class ReflectUtil {
               // to set candidateMethod = method
             } else {
               // c1 and c2 are not directly related
-              throw new IllegalArgumentException(
-                  "dispatch ambiguity between " + candidateMethod
-                  + " and " + method);
+              throw new IllegalArgumentException("dispatch ambiguity between "
+                  + candidateMethod + " and " + method);
             }
           }
         }
@@ -557,9 +561,8 @@ public abstract class ReflectUtil {
               new ArrayList<Class>();
           classList.add(arg0Clazz);
           classList.addAll(otherArgClassList);
-          throw new IllegalArgumentException(
-              "Method not found: " + methodName + "(" + classList
-              + ")");
+          throw new IllegalArgumentException("Method not found: " + methodName
+              + "(" + classList + ")");
         }
         return method;
       }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/ReflectiveVisitDispatcher.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/ReflectiveVisitDispatcher.java b/core/src/main/java/org/apache/calcite/util/ReflectiveVisitDispatcher.java
index a4d60a4..bfb444f 100644
--- a/core/src/main/java/org/apache/calcite/util/ReflectiveVisitDispatcher.java
+++ b/core/src/main/java/org/apache/calcite/util/ReflectiveVisitDispatcher.java
@@ -14,11 +14,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.util;
+package org.apache.calcite.util;
 
-import java.lang.reflect.*;
-
-import java.util.*;
+import java.lang.reflect.Method;
+import java.util.List;
 
 /**
  * Interface for looking up methods relating to reflective visitation. One
@@ -29,6 +28,9 @@ import java.util.*;
  *
  * <p>TODO: obsolete {@link ReflectUtil#lookupVisitMethod}, and use caching in
  * implementing that method.
+ *
+ * @param <E> Argument type
+ * @param <R> Return type
  */
 public interface ReflectiveVisitDispatcher<R extends ReflectiveVisitor, E> {
   //~ Methods ----------------------------------------------------------------
@@ -66,8 +68,8 @@ public interface ReflectiveVisitDispatcher<R extends ReflectiveVisitor, E> {
       String visitMethodName);
 
   /**
-   * Implements the {@link Glossary#VISITOR_PATTERN} via reflection. The basic
-   * technique is taken from <a
+   * Implements the {@link org.apache.calcite.util.Glossary#VISITOR_PATTERN} via
+   * reflection. The basic technique is taken from <a
    * href="http://www.javaworld.com/javaworld/javatips/jw-javatip98.html">a
    * Javaworld article</a>. For an example of how to use it, see
    * {@code ReflectVisitorTest}.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/ReflectiveVisitor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/ReflectiveVisitor.java b/core/src/main/java/org/apache/calcite/util/ReflectiveVisitor.java
index d5fa6e3..fa5d810 100644
--- a/core/src/main/java/org/apache/calcite/util/ReflectiveVisitor.java
+++ b/core/src/main/java/org/apache/calcite/util/ReflectiveVisitor.java
@@ -14,11 +14,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.util;
+package org.apache.calcite.util;
 
 /**
- * Object which can be a target for a reflective visitation (see {@link
- * ReflectUtil#invokeVisitor(ReflectiveVisitor, Object, Class, String)}.
+ * Object which can be a target for a reflective visitation (see
+ * {@link ReflectUtil#invokeVisitor(ReflectiveVisitor, Object, Class, String)}.
  *
  * <p>This is a tagging interface: it has no methods, and is not even required
  * in order to use reflective visitation, but serves to advise users of the

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/RhBase64.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/RhBase64.java b/core/src/main/java/org/apache/calcite/util/RhBase64.java
index bf56dc0..c24db2b 100644
--- a/core/src/main/java/org/apache/calcite/util/RhBase64.java
+++ b/core/src/main/java/org/apache/calcite/util/RhBase64.java
@@ -14,9 +14,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.util;
-
-import java.io.*;
+package org.apache.calcite.util;
+
+import java.io.BufferedInputStream;
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.FilterInputStream;
+import java.io.FilterOutputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.Serializable;
+import java.io.UnsupportedEncodingException;
 import java.util.zip.GZIPInputStream;
 import java.util.zip.GZIPOutputStream;
 
@@ -704,9 +716,8 @@ public class RhBase64 {
           }
         }
       } else {
-        System.err.println(
-            "Bad Base64 input character at " + i + ": " + source[i]
-            + "(decimal)");
+        System.err.println("Bad Base64 input character at " + i + ": "
+            + source[i] + "(decimal)");
         return null;
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/SaffronProperties.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/SaffronProperties.java b/core/src/main/java/org/apache/calcite/util/SaffronProperties.java
index dc5e9cd..88025d6 100644
--- a/core/src/main/java/org/apache/calcite/util/SaffronProperties.java
+++ b/core/src/main/java/org/apache/calcite/util/SaffronProperties.java
@@ -14,15 +14,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.util;
+package org.apache.calcite.util;
 
-import java.io.*;
+import org.eigenbase.util.property.BooleanProperty;
+import org.eigenbase.util.property.StringProperty;
 
-import java.security.*;
-
-import java.util.*;
-
-import org.eigenbase.util.property.*;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.security.AccessControlException;
+import java.util.Enumeration;
+import java.util.Properties;
 
 /**
  * Provides an environment for debugging information, et cetera, used by
@@ -39,9 +41,9 @@ import org.eigenbase.util.property.*;
  * </p>
  *
  * <p>Every property used in saffron code must have a member in this class. The
- * member must be public and final, and be of type {@link
- * org.eigenbase.util.property.Property} or some subtype. The javadoc comment
- * must describe the name of the property (for example,
+ * member must be public and final, and be of type
+ * {@link org.eigenbase.util.property.Property} or some subtype. The javadoc
+ * comment must describe the name of the property (for example,
  * "net.sf.saffron.connection.PoolSize") and the default value, if any. <em>
  * Developers, please make sure that this remains so!</em></p>
  */
@@ -69,8 +71,8 @@ public class SaffronProperties extends Properties {
 
   /**
    * The string property "saffron.default.charset" is the name of the default
-   * character set. The default is "ISO-8859-1". It is used in {@link
-   * org.eigenbase.sql.validate.SqlValidator}.
+   * character set. The default is "ISO-8859-1". It is used in
+   * {@link org.apache.calcite.sql.validate.SqlValidator}.
    */
   public final StringProperty defaultCharset =
       new StringProperty(this, "saffron.default.charset", "ISO-8859-1");
@@ -79,8 +81,8 @@ public class SaffronProperties extends Properties {
    * The string property "saffron.default.nationalcharset" is the name of the
    * default national character set which is used with the N'string' construct
    * which may or may not be different from the {@link #defaultCharset}. The
-   * default is "ISO-8859-1". It is used in {@link
-   * org.eigenbase.sql.SqlLiteral#SqlLiteral}
+   * default is "ISO-8859-1". It is used in
+   * {@link org.apache.calcite.sql.SqlLiteral#SqlLiteral}
    */
   public final StringProperty defaultNationalCharset =
       new StringProperty(
@@ -90,9 +92,9 @@ public class SaffronProperties extends Properties {
 
   /**
    * The string property "saffron.default.collation.name" is the name of the
-   * default collation. The default is "ISO-8859-1$en_US". Used in {@link
-   * org.eigenbase.sql.SqlCollation} and {@link
-   * org.eigenbase.sql.SqlLiteral#SqlLiteral}
+   * default collation. The default is "ISO-8859-1$en_US". Used in
+   * {@link org.apache.calcite.sql.SqlCollation} and
+   * {@link org.apache.calcite.sql.SqlLiteral#SqlLiteral}
    */
   public final StringProperty defaultCollation =
       new StringProperty(
@@ -102,9 +104,9 @@ public class SaffronProperties extends Properties {
 
   /**
    * The string property "saffron.default.collation.strength" is the strength
-   * of the default collation. The default is "primary". Used in {@link
-   * org.eigenbase.sql.SqlCollation} and {@link
-   * org.eigenbase.sql.SqlLiteral#SqlLiteral}
+   * of the default collation. The default is "primary". Used in
+   * {@link org.apache.calcite.sql.SqlCollation} and
+   * {@link org.apache.calcite.sql.SqlLiteral#SqlLiteral}
    */
   public final StringProperty defaultCollationStrength =
       new StringProperty(

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/SerializableCharset.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/SerializableCharset.java b/core/src/main/java/org/apache/calcite/util/SerializableCharset.java
index 87efce3..0c6ca81 100644
--- a/core/src/main/java/org/apache/calcite/util/SerializableCharset.java
+++ b/core/src/main/java/org/apache/calcite/util/SerializableCharset.java
@@ -14,11 +14,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.util;
+package org.apache.calcite.util;
 
-import java.io.*;
-
-import java.nio.charset.*;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.Serializable;
+import java.nio.charset.Charset;
 
 /**
  * Serializable wrapper around a {@link Charset}.
@@ -38,8 +40,8 @@ public class SerializableCharset implements Serializable {
   //~ Constructors -----------------------------------------------------------
 
   /**
-   * Creates a SerializableCharset. External users should call {@link
-   * #forCharset(Charset)}.
+   * Creates a SerializableCharset. External users should call
+   * {@link #forCharset(Charset)}.
    *
    * @param charset Character set; must not be null
    */

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/StackWriter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/StackWriter.java b/core/src/main/java/org/apache/calcite/util/StackWriter.java
index 72f1dae..da2295c 100644
--- a/core/src/main/java/org/apache/calcite/util/StackWriter.java
+++ b/core/src/main/java/org/apache/calcite/util/StackWriter.java
@@ -14,11 +14,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.util;
+package org.apache.calcite.util;
 
-import java.io.*;
-
-import java.util.*;
+import java.io.FilterWriter;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.io.Writer;
+import java.util.ArrayList;
+import java.util.List;
 
 /**
  * A helper class for generating formatted text. StackWriter keeps track of
@@ -112,8 +115,8 @@ public class StackWriter extends FilterWriter {
    * specified string to be used for each level of indentation.
    *
    * @param writer      underlying writer
-   * @param indentation indentation unit such as {@link #INDENT_TAB} or {@link
-   *                    #INDENT_SPACE4}
+   * @param indentation indentation unit such as {@link #INDENT_TAB} or
+   *                    {@link #INDENT_SPACE4}
    */
   public StackWriter(Writer writer, String indentation) {
     super(writer);

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/Stacks.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/Stacks.java b/core/src/main/java/org/apache/calcite/util/Stacks.java
index cee92cc..70ef045 100644
--- a/core/src/main/java/org/apache/calcite/util/Stacks.java
+++ b/core/src/main/java/org/apache/calcite/util/Stacks.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.util;
+package org.apache.calcite.util;
 
 import java.util.List;
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/Static.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/Static.java b/core/src/main/java/org/apache/calcite/util/Static.java
index 9e5030b..a761d80 100644
--- a/core/src/main/java/org/apache/calcite/util/Static.java
+++ b/core/src/main/java/org/apache/calcite/util/Static.java
@@ -14,28 +14,38 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.util;
+package org.apache.calcite.util;
 
-import org.eigenbase.resource.EigenbaseNewResource;
-import org.eigenbase.resource.Resources;
+import org.apache.calcite.runtime.CalciteResource;
+
+import org.apache.calcite.runtime.Resources;
 
 /**
  * Definitions of objects to be statically imported.
  *
- * <p>Developers: Give careful consideration before including an object in this
- * class.
- * Pros: Code that uses these objects will be terser.
- * Cons: Namespace pollution,
- * code that is difficult to understand (a general problem with static imports),
- * potential cyclic initialization.</p>
+ * <h3>Note to developers</h3>
+ *
+ * <p>Please give careful consideration before including an object in this
+ * class. Pros:
+ * <ul>
+ * <li>Code that uses these objects will be terser.
+ * </ul>
+ *
+ * <p>Cons:</p>
+ * <ul>
+ * <li>Namespace pollution,
+ * <li>code that is difficult to understand (a general problem with static
+ * imports),
+ * <li>potential cyclic initialization.
+ * </ul>
  */
 public abstract class Static {
   private Static() {}
 
   /** Resources. */
-  public static final EigenbaseNewResource RESOURCE =
-      Resources.create("org.eigenbase.resource.EigenbaseResource",
-          EigenbaseNewResource.class);
+  public static final CalciteResource RESOURCE =
+      Resources.create("org.apache.calcite.runtime.CalciteResource",
+          CalciteResource.class);
 }
 
 // End Static.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/Template.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/Template.java b/core/src/main/java/org/apache/calcite/util/Template.java
index d7e1737..1936f8e 100644
--- a/core/src/main/java/org/apache/calcite/util/Template.java
+++ b/core/src/main/java/org/apache/calcite/util/Template.java
@@ -14,13 +14,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.util;
-
-import java.text.MessageFormat;
-import java.util.*;
+package org.apache.calcite.util;
 
 import com.google.common.collect.ImmutableList;
 
+import java.text.MessageFormat;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+
 /**
  * String template.
  *

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/UnmodifiableArrayList.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/UnmodifiableArrayList.java b/core/src/main/java/org/apache/calcite/util/UnmodifiableArrayList.java
index 119df37..3a27b02 100644
--- a/core/src/main/java/org/apache/calcite/util/UnmodifiableArrayList.java
+++ b/core/src/main/java/org/apache/calcite/util/UnmodifiableArrayList.java
@@ -14,13 +14,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.util;
+package org.apache.calcite.util;
+
+import com.google.common.base.Preconditions;
 
 import java.util.AbstractList;
 import java.util.RandomAccess;
 
-import com.google.common.base.Preconditions;
-
 /**
  * A view onto an array that cannot be modified by the client.
  *

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/Util.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/Util.java b/core/src/main/java/org/apache/calcite/util/Util.java
index 41587a6..435df11 100644
--- a/core/src/main/java/org/apache/calcite/util/Util.java
+++ b/core/src/main/java/org/apache/calcite/util/Util.java
@@ -14,27 +14,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.util;
+package org.apache.calcite.util;
 
-import java.awt.Toolkit;
-import java.io.*;
-import java.lang.reflect.Array;
-import java.lang.reflect.Field;
-import java.math.*;
-import java.net.*;
-import java.nio.charset.*;
-import java.sql.*;
-import java.text.*;
-import java.util.*;
-import java.util.jar.*;
-import java.util.logging.*;
-import java.util.regex.*;
-
-import javax.annotation.Nullable;
-
-import net.hydromatic.linq4j.Ord;
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.runtime.CalciteException;
 
 import com.google.common.base.Function;
+import com.google.common.base.Objects;
 import com.google.common.cache.CacheBuilder;
 import com.google.common.cache.CacheLoader;
 import com.google.common.cache.LoadingCache;
@@ -42,6 +28,52 @@ import com.google.common.collect.Collections2;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.PrintWriter;
+import java.io.Reader;
+import java.io.StringWriter;
+import java.io.Writer;
+import java.lang.reflect.Array;
+import java.lang.reflect.Field;
+import java.math.BigDecimal;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.nio.charset.Charset;
+import java.sql.Connection;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.text.SimpleDateFormat;
+import java.util.AbstractCollection;
+import java.util.AbstractList;
+import java.util.AbstractMap;
+import java.util.AbstractSet;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Enumeration;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Hashtable;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.TimeZone;
+import java.util.jar.JarFile;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import javax.annotation.Nullable;
+
 /**
  * Miscellaneous utility functions.
  */
@@ -51,17 +83,6 @@ public class Util {
   //~ Static fields/initializers ---------------------------------------------
 
   /**
-   * Name of the system property that controls whether the AWT work-around is
-   * enabled. This workaround allows Farrago to load its native libraries
-   * despite a conflict with AWT and allows applications that use AWT to
-   * function normally.
-   *
-   * @see #loadLibrary(String)
-   */
-  public static final String AWT_WORKAROUND_PROPERTY =
-      "org.eigenbase.util.AWT_WORKAROUND";
-
-  /**
    * System-dependent newline character.
    *
    * <p>In general, you should not use this in expected results of tests.
@@ -84,8 +105,6 @@ public class Util {
    */
   public static final String FILE_TIMESTAMP_FORMAT = "yyyy-MM-dd_HH_mm_ss";
 
-  private static boolean driversLoaded = false;
-
   /**
    * Regular expression for a valid java identifier which contains no
    * underscores and can therefore be returned intact by {@link #toJavaId}.
@@ -94,11 +113,6 @@ public class Util {
       Pattern.compile("[a-zA-Z_$][a-zA-Z0-9$]*");
 
   /**
-   * @see #loadLibrary(String)
-   */
-  private static Toolkit awtToolkit;
-
-  /**
    * Maps classes to the map of their enum values. Uses a weak map so that
    * classes are not prevented from being unloaded.
    */
@@ -107,8 +121,7 @@ public class Util {
           .weakKeys()
           .build(
               new CacheLoader<Class, Map<String, Enum>>() {
-                @Override
-                public Map<String, Enum> load(Class clazz) {
+                @Override public Map<String, Enum> load(Class clazz) {
                   //noinspection unchecked
                   return enumConstants(clazz);
                 }
@@ -174,21 +187,6 @@ public class Util {
   }
 
   /**
-   * Returns whether two objects are equal or are both null.
-   */
-  public static boolean equal(
-      Object s0,
-      Object s1) {
-    if (s0 == s1) {
-      return true;
-    } else if (s0 == null) {
-      return false;
-    } else {
-      return s0.equals(s1);
-    }
-  }
-
-  /**
    * Returns whether two lists are equal to each other using shallow
    * comparisons.
    *
@@ -251,17 +249,6 @@ public class Util {
     return h;
   }
 
-  /**
-   * Computes a hash code over var args.
-   */
-  public static int hashV(Object... a) {
-    int h = 19690721;
-    for (Object o : a) {
-      h = hash(h, o);
-    }
-    return h;
-  }
-
   /** Computes the hash code of a {@code double} value. Equivalent to
    * {@link Double}{@code .hashCode(double)}, but that method was only
    * introduced in JDK 1.8.
@@ -655,8 +642,8 @@ public class Util {
   }
 
   /**
-   * Materializes the results of a {@link java.util.Iterator} as a {@link
-   * java.util.List}.
+   * Materializes the results of a {@link java.util.Iterator} as a
+   * {@link java.util.List}.
    *
    * @param iter iterator to materialize
    * @return materialized list
@@ -715,8 +702,8 @@ public class Util {
   }
 
   /**
-   * Returns the {@link Charset} object representing the value of {@link
-   * SaffronProperties#defaultCharset}
+   * Returns the {@link Charset} object representing the value of
+   * {@link SaffronProperties#defaultCharset}
    *
    * @throws java.nio.charset.IllegalCharsetNameException If the given charset
    *                                                      name is illegal
@@ -765,7 +752,7 @@ public class Util {
     StringBuilder sb = new StringBuilder();
     for (Throwable curr = t; curr != null; curr = curr.getCause()) {
       String msg =
-          ((curr instanceof EigenbaseException)
+          ((curr instanceof CalciteException)
               || (curr instanceof SQLException)) ? curr.getMessage()
               : curr.toString();
       if (sb.length() > 0) {
@@ -944,46 +931,6 @@ public class Util {
   }
 
   /**
-   * Uses {@link System#loadLibrary(String)} to load a native library
-   * correctly under mingw (Windows/Cygwin) and Linux environments.
-   *
-   * <p>This method also implements a work-around for applications that wish
-   * to load AWT. AWT conflicts with some native libraries in a way that
-   * requires AWT to be loaded first. This method checks the system property
-   * named {@link #AWT_WORKAROUND_PROPERTY} and if it is set to "on" (default;
-   * case-insensitive) it pre-loads AWT to avoid the conflict.
-   *
-   * @param libName the name of the library to load, as in {@link
-   *                System#loadLibrary(String)}.
-   */
-  public static void loadLibrary(String libName) {
-    String awtSetting = System.getProperty(AWT_WORKAROUND_PROPERTY, "on");
-    if ((awtToolkit == null) && awtSetting.equalsIgnoreCase("on")) {
-      // REVIEW jvs 8-Sept-2006:  workaround upon workaround.  This
-      // is required because in native code, we sometimes (see Farrago)
-      // have to use dlopen("libfoo.so", RTLD_GLOBAL) in order for native
-      // plugins to load correctly.  But the RTLD_GLOBAL causes trouble
-      // later if someone tries to use AWT from within the same JVM.
-      // So... preload AWT here unless someone configured explicitly
-      // not to do so.
-      try {
-        awtToolkit = Toolkit.getDefaultToolkit();
-      } catch (Throwable ex) {
-        // Suppress problems so that a headless server doesn't fail on
-        // startup.  If AWT is actually needed, the same exception will
-        // show up later, which is fine.
-
-        // NOTE jvs 27-Mar-2007: If this exception occurs, we'll
-        // retry the AWT load on each loadLibrary call.  That's okay,
-        // since there are only a few libraries and they're loaded
-        // via static initializers.
-      }
-    }
-
-    System.loadLibrary(libName);
-  }
-
-  /**
    * Returns whether an array of strings contains a given string among the
    * first <code>length</code> entries.
    *
@@ -1282,8 +1229,7 @@ public class Util {
       // POSIX allows us to omit DST offset if it is 1:00:00
       appendPosixTime(buf, dstSavings);
     }
-    String patternString =
-        ".*,"
+    String patternString = ".*,"
         + "startMode=([0-9]*),"
         + "startMonth=([0-9]*),"
         + "startDay=([-0-9]*),"
@@ -1545,7 +1491,8 @@ public class Util {
   /**
    * Runs an external application process.
    *
-   * @param pb        {@link ProcessBuilder} for the application; might be returned by {@link #newAppProcess}.
+   * @param pb        ProcessBuilder for the application; might be
+   *                  returned by {@link #newAppProcess}.
    * @param logger    if not null, command and exit status will be logged here
    * @param appInput  if not null, data will be copied to application's stdin
    * @param appOutput if not null, data will be captured from application's
@@ -1611,8 +1558,8 @@ public class Util {
    * type.
    *
    * <p>If a member of the backing list is not an instanceof <code>E</code>,
-   * the accessing method (such as {@link List#get}) will throw a {@link
-   * ClassCastException}.
+   * the accessing method (such as {@link List#get}) will throw a
+   * {@link ClassCastException}.
    *
    * <p>All modifications are automatically written to the backing list. Not
    * synchronized.
@@ -1630,8 +1577,8 @@ public class Util {
    * type.
    *
    * <p>If a member of the backing iterator is not an instanceof <code>
-   * E</code>, {@link Iterator#next()}) will throw a {@link
-   * ClassCastException}.
+   * E</code>, {@link Iterator#next()}) will throw a
+   * {@link ClassCastException}.
    *
    * <p>All modifications are automatically written to the backing iterator.
    * Not synchronized.
@@ -1876,8 +1823,7 @@ public class Util {
    * @return an error, to be thrown
    */
   public static <E extends Enum<E>> Error unexpected(E value) {
-    return new AssertionError(
-        "Was not expecting value '" + value
+    return new AssertionError("Was not expecting value '" + value
         + "' for enumeration '" + value.getDeclaringClass().getName()
         + "' in this context");
   }
@@ -2015,48 +1961,6 @@ public class Util {
   }
 
   /**
-   * Converts an underscore-separated name into a camelCase name.
-   * For example, {@code uncamel("MY_JDBC_DRIVER")} returns "myJdbcDriver".
-   */
-  public static String toCamelCase(String name) {
-    StringBuilder buf = new StringBuilder();
-    int nextUpper = -1;
-    for (int i = 0; i < name.length(); i++) {
-      char c = name.charAt(i);
-      if (c == '_') {
-        nextUpper = i + 1;
-        continue;
-      }
-      if (nextUpper == i) {
-        c = Character.toUpperCase(c);
-      } else {
-        c = Character.toLowerCase(c);
-      }
-      buf.append(c);
-    }
-    return buf.toString();
-  }
-
-  /**
-   * Converts a camelCase name into an upper-case underscore-separated name.
-   * For example, {@code camelToUpper("myJdbcDriver")} returns
-   * "MY_JDBC_DRIVER".
-   */
-  public static String camelToUpper(String name) {
-    StringBuilder buf = new StringBuilder();
-    for (int i = 0; i < name.length(); i++) {
-      char c = name.charAt(i);
-      if (Character.isUpperCase(c)) {
-        buf.append('_');
-      } else {
-        c = Character.toUpperCase(c);
-      }
-      buf.append(c);
-    }
-    return buf.toString();
-  }
-
-  /**
    * Returns whether the elements of {@code list} are distinct.
    */
   public static <E> boolean isDistinct(List<E> list) {
@@ -2086,7 +1990,7 @@ public class Util {
         E e = list.get(i);
         for (int j = i - 1; j >= 0; j--) {
           E e1 = list.get(j);
-          if (equal(e, e1)) {
+          if (Objects.equal(e, e1)) {
             return i;
           }
         }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/XmlOutput.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/XmlOutput.java b/core/src/main/java/org/apache/calcite/util/XmlOutput.java
index cd529e8..4be5669 100644
--- a/core/src/main/java/org/apache/calcite/util/XmlOutput.java
+++ b/core/src/main/java/org/apache/calcite/util/XmlOutput.java
@@ -14,13 +14,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.util;
-
-import java.io.*;
-import java.util.*;
+package org.apache.calcite.util;
 
 import com.google.common.collect.Lists;
 
+import java.io.IOException;
+import java.io.LineNumberReader;
+import java.io.PrintWriter;
+import java.io.StringReader;
+import java.io.Writer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
 /**
  * Streaming XML output.
  *

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/ZonelessDate.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/ZonelessDate.java b/core/src/main/java/org/apache/calcite/util/ZonelessDate.java
index 80cfa79..9648327 100644
--- a/core/src/main/java/org/apache/calcite/util/ZonelessDate.java
+++ b/core/src/main/java/org/apache/calcite/util/ZonelessDate.java
@@ -14,12 +14,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.util14;
+package org.apache.calcite.util;
 
 import java.sql.Date;
-
-import java.text.*;
-
+import java.text.DateFormat;
 import java.util.Calendar;
 import java.util.TimeZone;
 
@@ -77,9 +75,10 @@ public class ZonelessDate extends ZonelessDatetime {
   }
 
   /**
-   * Formats this ZonelessDate via a SimpleDateFormat
+   * Formats this ZonelessDate via a SimpleDateFormat.
    *
-   * @param format format string, as required by {@link SimpleDateFormat}
+   * @param format Format string, as required by
+   *     {@link java.text.SimpleDateFormat}
    * @return the formatted date string
    */
   public String toString(String format) {
@@ -104,7 +103,7 @@ public class ZonelessDate extends ZonelessDatetime {
    *
    * @param s      a string representing a date in ISO format, i.e. according to
    *               the SimpleDateFormat string "yyyy-MM-dd"
-   * @param format format string as per {@link SimpleDateFormat}
+   * @param format Format string as per {@link java.text.SimpleDateFormat}
    * @return the parsed date, or null if parsing failed
    */
   public static ZonelessDate parse(String s, String format) {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/ZonelessDatetime.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/ZonelessDatetime.java b/core/src/main/java/org/apache/calcite/util/ZonelessDatetime.java
index 7f030f0..abe8165 100644
--- a/core/src/main/java/org/apache/calcite/util/ZonelessDatetime.java
+++ b/core/src/main/java/org/apache/calcite/util/ZonelessDatetime.java
@@ -14,13 +14,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.util14;
+package org.apache.calcite.util;
 
-import java.io.*;
-
-import java.text.*;
-
-import java.util.*;
+import java.io.Serializable;
+import java.text.DateFormat;
+import java.util.Calendar;
+import java.util.TimeZone;
 
 /**
  * ZonelessDatetime is an abstract class for dates, times, or timestamps that
@@ -101,8 +100,8 @@ public abstract class ZonelessDatetime implements BasicDatetime, Serializable {
   }
 
   /**
-   * Gets the value of this datetime as a milliseconds value for {@link
-   * java.sql.Time}.
+   * Gets the value of this datetime as a milliseconds value for
+   * {@link java.sql.Time}.
    *
    * @param zone time zone in which to generate a time value for
    */
@@ -112,8 +111,8 @@ public abstract class ZonelessDatetime implements BasicDatetime, Serializable {
   }
 
   /**
-   * Gets the value of this datetime as a milliseconds value for {@link
-   * java.sql.Date}.
+   * Gets the value of this datetime as a milliseconds value for
+   * {@link java.sql.Date}.
    *
    * @param zone time zone in which to generate a time value for
    */
@@ -132,8 +131,8 @@ public abstract class ZonelessDatetime implements BasicDatetime, Serializable {
   }
 
   /**
-   * Gets the value of this datetime as a milliseconds value for {@link
-   * java.sql.Timestamp}.
+   * Gets the value of this datetime as a milliseconds value for
+   * {@link java.sql.Timestamp}.
    *
    * @param zone time zone in which to generate a time value for
    */

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/ZonelessTime.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/ZonelessTime.java b/core/src/main/java/org/apache/calcite/util/ZonelessTime.java
index 5a4b52b..d55cb70 100644
--- a/core/src/main/java/org/apache/calcite/util/ZonelessTime.java
+++ b/core/src/main/java/org/apache/calcite/util/ZonelessTime.java
@@ -14,12 +14,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.util14;
+package org.apache.calcite.util;
 
 import java.sql.Time;
-
-import java.text.*;
-
+import java.text.DateFormat;
+import java.text.SimpleDateFormat;
 import java.util.Calendar;
 import java.util.TimeZone;
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/ZonelessTimestamp.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/ZonelessTimestamp.java b/core/src/main/java/org/apache/calcite/util/ZonelessTimestamp.java
index 0f99596..94a6b76 100644
--- a/core/src/main/java/org/apache/calcite/util/ZonelessTimestamp.java
+++ b/core/src/main/java/org/apache/calcite/util/ZonelessTimestamp.java
@@ -14,11 +14,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.util14;
+package org.apache.calcite.util;
 
-import java.sql.*;
-
-import java.text.*;
+import java.sql.Timestamp;
+import java.text.DateFormat;
 
 /**
  * ZonelessTimestamp is a timestamp value without a time zone.
@@ -122,7 +121,7 @@ public class ZonelessTimestamp extends ZonelessDatetime {
    *
    * @param s      a string representing a time in ISO format, i.e. according to
    *               the SimpleDateFormat string "yyyy-MM-dd HH:mm:ss"
-   * @param format format string as per {@link SimpleDateFormat}
+   * @param format Format string as per {@link java.text.SimpleDateFormat}
    * @return the parsed timestamp, or null if parsing failed
    */
   public static ZonelessTimestamp parse(String s, String format) {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/graph/BreadthFirstIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/graph/BreadthFirstIterator.java b/core/src/main/java/org/apache/calcite/util/graph/BreadthFirstIterator.java
index 0f7589b..21aed24 100644
--- a/core/src/main/java/org/apache/calcite/util/graph/BreadthFirstIterator.java
+++ b/core/src/main/java/org/apache/calcite/util/graph/BreadthFirstIterator.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.util.graph;
+package org.apache.calcite.util.graph;
 
 import java.util.ArrayDeque;
 import java.util.Deque;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/graph/CycleDetector.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/graph/CycleDetector.java b/core/src/main/java/org/apache/calcite/util/graph/CycleDetector.java
index 8480435..4e9fcc5 100644
--- a/core/src/main/java/org/apache/calcite/util/graph/CycleDetector.java
+++ b/core/src/main/java/org/apache/calcite/util/graph/CycleDetector.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.util.graph;
+package org.apache.calcite.util.graph;
 
 import java.util.Set;
 
@@ -37,4 +37,3 @@ public class CycleDetector<V, E extends DefaultEdge> {
 }
 
 // End CycleDetector.java
-

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/graph/DefaultDirectedGraph.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/graph/DefaultDirectedGraph.java b/core/src/main/java/org/apache/calcite/util/graph/DefaultDirectedGraph.java
index 3bafdf6..f093393 100644
--- a/core/src/main/java/org/apache/calcite/util/graph/DefaultDirectedGraph.java
+++ b/core/src/main/java/org/apache/calcite/util/graph/DefaultDirectedGraph.java
@@ -14,9 +14,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.util.graph;
-
-import java.util.*;
+package org.apache.calcite.util.graph;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
 
 /**
  * Default implementation of {@link DirectedGraph}.
@@ -45,8 +53,7 @@ public class DefaultDirectedGraph<V, E extends DefaultEdge>
     return new DefaultDirectedGraph<V, E>(edgeFactory);
   }
 
-  @Override
-  public String toString() {
+  @Override public String toString() {
     StringBuilder buf = new StringBuilder();
     buf.append("graph(")
         .append("vertices: ")

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/graph/DefaultEdge.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/graph/DefaultEdge.java b/core/src/main/java/org/apache/calcite/util/graph/DefaultEdge.java
index b44cf16..3f32bdf 100644
--- a/core/src/main/java/org/apache/calcite/util/graph/DefaultEdge.java
+++ b/core/src/main/java/org/apache/calcite/util/graph/DefaultEdge.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.util.graph;
+package org.apache.calcite.util.graph;
 
 /**
  * Default implementation of Edge.
@@ -28,13 +28,11 @@ public class DefaultEdge {
     this.target = target;
   }
 
-  @Override
-  public int hashCode() {
+  @Override public int hashCode() {
     return source.hashCode() * 31 + target.hashCode();
   }
 
-  @Override
-  public boolean equals(Object obj) {
+  @Override public boolean equals(Object obj) {
     return this == obj
         || obj instanceof DefaultEdge
         && ((DefaultEdge) obj).source.equals(source)

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/graph/DepthFirstIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/graph/DepthFirstIterator.java b/core/src/main/java/org/apache/calcite/util/graph/DepthFirstIterator.java
index c6cd7cf..1631d38 100644
--- a/core/src/main/java/org/apache/calcite/util/graph/DepthFirstIterator.java
+++ b/core/src/main/java/org/apache/calcite/util/graph/DepthFirstIterator.java
@@ -14,12 +14,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.util.graph;
+package org.apache.calcite.util.graph;
 
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
 
-import java.util.*;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
 
 /**
  * Iterates over the vertices in a directed graph in depth-first order.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/graph/DirectedGraph.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/graph/DirectedGraph.java b/core/src/main/java/org/apache/calcite/util/graph/DirectedGraph.java
index b7b9939..ba5c69c 100644
--- a/core/src/main/java/org/apache/calcite/util/graph/DirectedGraph.java
+++ b/core/src/main/java/org/apache/calcite/util/graph/DirectedGraph.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.util.graph;
+package org.apache.calcite.util.graph;
 
 import java.util.Collection;
 import java.util.List;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/graph/Graphs.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/graph/Graphs.java b/core/src/main/java/org/apache/calcite/util/graph/Graphs.java
index cb5f657..c1b94a6 100644
--- a/core/src/main/java/org/apache/calcite/util/graph/Graphs.java
+++ b/core/src/main/java/org/apache/calcite/util/graph/Graphs.java
@@ -14,13 +14,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.util.graph;
+package org.apache.calcite.util.graph;
 
-import org.eigenbase.util.Pair;
+import org.apache.calcite.util.Pair;
 
 import com.google.common.collect.ImmutableList;
 
-import java.util.*;
+import java.util.AbstractList;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
 
 /**
  * Miscellaneous graph utilities.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/graph/TopologicalOrderIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/graph/TopologicalOrderIterator.java b/core/src/main/java/org/apache/calcite/util/graph/TopologicalOrderIterator.java
index da4d537..db7154f 100644
--- a/core/src/main/java/org/apache/calcite/util/graph/TopologicalOrderIterator.java
+++ b/core/src/main/java/org/apache/calcite/util/graph/TopologicalOrderIterator.java
@@ -14,9 +14,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.util.graph;
+package org.apache.calcite.util.graph;
 
-import java.util.*;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
 
 /**
  * Iterates over the edges of a graph in topological order.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/graph/package-info.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/graph/package-info.java b/core/src/main/java/org/apache/calcite/util/graph/package-info.java
index d2a0024..69d3018 100644
--- a/core/src/main/java/org/apache/calcite/util/graph/package-info.java
+++ b/core/src/main/java/org/apache/calcite/util/graph/package-info.java
@@ -18,6 +18,6 @@
 /**
  * Graph-theoretic algorithms and data structures.
  */
-package net.hydromatic.optiq.util.graph;
+package org.apache.calcite.util.graph;
 
 // End package-info.java


[21/58] [abbrv] [partial] incubator-calcite git commit: [CALCITE-306] Standardize code style for "import package.*; "

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql2rel/SqlRexConvertletTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql2rel/SqlRexConvertletTable.java b/core/src/main/java/org/apache/calcite/sql2rel/SqlRexConvertletTable.java
index 09a6a75..b1e0a51 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/SqlRexConvertletTable.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/SqlRexConvertletTable.java
@@ -14,9 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql2rel;
+package org.apache.calcite.sql2rel;
 
-import org.eigenbase.sql.*;
+import org.apache.calcite.sql.SqlCall;
 
 /**
  * Collection of {@link SqlRexConvertlet}s.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java b/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
index c940b6c..ce1cb4a 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
@@ -14,47 +14,160 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql2rel;
-
-import java.lang.reflect.Type;
-import java.math.*;
-import java.util.*;
-import java.util.logging.*;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.rel.metadata.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.rex.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.fun.*;
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.sql.util.*;
-import org.eigenbase.sql.validate.*;
-import org.eigenbase.trace.*;
-import org.eigenbase.util.*;
-import org.eigenbase.util.mapping.Mappings;
-import org.eigenbase.util14.*;
-
-import net.hydromatic.linq4j.Ord;
-
-import net.hydromatic.optiq.ModifiableTable;
-import net.hydromatic.optiq.TranslatableTable;
-import net.hydromatic.optiq.prepare.Prepare;
-import net.hydromatic.optiq.prepare.RelOptTableImpl;
-import net.hydromatic.optiq.util.BitSets;
+package org.apache.calcite.sql2rel;
+
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelOptQuery;
+import org.apache.calcite.plan.RelOptSamplingParameters;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.prepare.Prepare;
+import org.apache.calcite.prepare.RelOptTableImpl;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollationImpl;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.core.Collect;
+import org.apache.calcite.rel.core.Correlation;
+import org.apache.calcite.rel.core.Correlator;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.JoinInfo;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.Sample;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.core.Uncollect;
+import org.apache.calcite.rel.logical.LogicalAggregate;
+import org.apache.calcite.rel.logical.LogicalIntersect;
+import org.apache.calcite.rel.logical.LogicalJoin;
+import org.apache.calcite.rel.logical.LogicalMinus;
+import org.apache.calcite.rel.logical.LogicalOneRow;
+import org.apache.calcite.rel.logical.LogicalProject;
+import org.apache.calcite.rel.logical.LogicalTableFunctionScan;
+import org.apache.calcite.rel.logical.LogicalTableModify;
+import org.apache.calcite.rel.logical.LogicalTableScan;
+import org.apache.calcite.rel.logical.LogicalUnion;
+import org.apache.calcite.rel.logical.LogicalValues;
+import org.apache.calcite.rel.metadata.RelColumnMapping;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexCallBinding;
+import org.apache.calcite.rex.RexCorrelVariable;
+import org.apache.calcite.rex.RexDynamicParam;
+import org.apache.calcite.rex.RexFieldAccess;
+import org.apache.calcite.rex.RexFieldCollation;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexRangeRef;
+import org.apache.calcite.rex.RexShuttle;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.rex.RexWindowBound;
+import org.apache.calcite.schema.ModifiableTable;
+import org.apache.calcite.schema.TranslatableTable;
+import org.apache.calcite.sql.JoinConditionType;
+import org.apache.calcite.sql.JoinType;
+import org.apache.calcite.sql.SqlAggFunction;
+import org.apache.calcite.sql.SqlBasicCall;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlDataTypeSpec;
+import org.apache.calcite.sql.SqlDelete;
+import org.apache.calcite.sql.SqlDynamicParam;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.SqlFunction;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlInsert;
+import org.apache.calcite.sql.SqlIntervalQualifier;
+import org.apache.calcite.sql.SqlJoin;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlMerge;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlNodeList;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlOperatorTable;
+import org.apache.calcite.sql.SqlSampleSpec;
+import org.apache.calcite.sql.SqlSelect;
+import org.apache.calcite.sql.SqlSelectKeyword;
+import org.apache.calcite.sql.SqlSetOperator;
+import org.apache.calcite.sql.SqlUpdate;
+import org.apache.calcite.sql.SqlUtil;
+import org.apache.calcite.sql.SqlWindow;
+import org.apache.calcite.sql.SqlWith;
+import org.apache.calcite.sql.SqlWithItem;
+import org.apache.calcite.sql.fun.SqlCountAggFunction;
+import org.apache.calcite.sql.fun.SqlInOperator;
+import org.apache.calcite.sql.fun.SqlRowOperator;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.SqlReturnTypeInference;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.type.SqlTypeUtil;
+import org.apache.calcite.sql.type.TableFunctionReturnTypeInference;
+import org.apache.calcite.sql.util.SqlBasicVisitor;
+import org.apache.calcite.sql.util.SqlVisitor;
+import org.apache.calcite.sql.validate.CollectNamespace;
+import org.apache.calcite.sql.validate.DelegatingScope;
+import org.apache.calcite.sql.validate.ListScope;
+import org.apache.calcite.sql.validate.ParameterScope;
+import org.apache.calcite.sql.validate.SelectScope;
+import org.apache.calcite.sql.validate.SqlMonotonicity;
+import org.apache.calcite.sql.validate.SqlUserDefinedTableFunction;
+import org.apache.calcite.sql.validate.SqlUserDefinedTableMacro;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql.validate.SqlValidatorImpl;
+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.BitSets;
+import org.apache.calcite.util.ImmutableIntList;
+import org.apache.calcite.util.NlsString;
+import org.apache.calcite.util.NumberUtil;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
+import org.apache.calcite.util.mapping.Mappings;
+import org.apache.calcite.util.trace.CalciteTrace;
 
 import com.google.common.base.Function;
-import com.google.common.collect.*;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
 
-import static org.eigenbase.sql.SqlUtil.stripAs;
-import static org.eigenbase.util.Static.RESOURCE;
+import java.lang.reflect.Type;
+import java.math.BigDecimal;
+import java.util.AbstractList;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.Stack;
+import java.util.TreeSet;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import static org.apache.calcite.sql.SqlUtil.stripAs;
+import static org.apache.calcite.util.Static.RESOURCE;
 
 /**
- * Converts a SQL parse tree (consisting of {@link org.eigenbase.sql.SqlNode}
- * objects) into a relational algebra expression (consisting of
- * {@link org.eigenbase.rel.RelNode} objects).
+ * Converts a SQL parse tree (consisting of
+ * {@link org.apache.calcite.sql.SqlNode} objects) into a relational algebra
+ * expression (consisting of {@link org.apache.calcite.rel.RelNode} objects).
  *
  * <p>The public entry points are: {@link #convertQuery},
  * {@link #convertExpression(SqlNode)}.
@@ -63,7 +176,7 @@ public class SqlToRelConverter {
   //~ Static fields/initializers ---------------------------------------------
 
   protected static final Logger SQL2REL_LOGGER =
-      EigenbaseTrace.getSqlToRelTracer();
+      CalciteTrace.getSqlToRelTracer();
 
   private static final Function<SubQuery, SqlNode> FN =
       new Function<SubQuery, SqlNode>() {
@@ -273,19 +386,20 @@ public class SqlToRelConverter {
    * convert method.
    *
    * @param enabled true for immediate conversion (the default); false to
-   *                generate logical TableAccessRel instances
+   *                generate logical LogicalTableScan instances
    */
   public void enableTableAccessConversion(boolean enabled) {
     shouldConvertTableAccess = enabled;
   }
 
   /**
-   * Controls whether instances of {@link ValuesRel} are generated. These may
-   * not be supported by all physical implementations. To have any effect,
+   * Controls whether instances of
+   * {@link org.apache.calcite.rel.logical.LogicalValues} are generated. These
+   * may not be supported by all physical implementations. To have any effect,
    * this must be called before any convert method.
    *
-   * @param enabled true to allow ValuesRel to be generated (the default);
-   *                false to force substitution of ProjectRel+OneRowRel instead
+   * @param enabled true to allow LogicalValues to be generated (the default);
+   *                false to force substitution of Project+OneRow instead
    */
   public void enableValuesRelCreation(boolean enabled) {
     shouldCreateValuesRel = enabled;
@@ -303,9 +417,8 @@ public class SqlToRelConverter {
         RelDataType validatedRowType =
             validator.getValidatedNodeType(query);
         validatedRowType = uniquifyFields(validatedRowType);
-        throw Util.newInternal(
-            "Conversion to relational algebra failed to preserve "
-            + "datatypes:\n"
+        throw Util.newInternal("Conversion to relational algebra failed to "
+            + "preserve datatypes:\n"
             + "validated type:\n"
             + validatedRowType.getFullTypeString()
             + "\nconverted type:\n"
@@ -512,10 +625,10 @@ public class SqlToRelConverter {
    * Having translated 'SELECT ... FROM ... [GROUP BY ...] [HAVING ...]', adds
    * a relational expression to make the results unique.
    *
-   * <p>If the SELECT clause contains duplicate expressions, adds {@link
-   * ProjectRel}s so that we are grouping on the minimal set of keys. The
-   * performance gain isn't huge, but it is difficult to detect these
-   * duplicate expressions later.
+   * <p>If the SELECT clause contains duplicate expressions, adds
+   * {@link org.apache.calcite.rel.logical.LogicalProject}s so that we are
+   * grouping on the minimal set of keys. The performance gain isn't huge, but
+   * it is difficult to detect these duplicate expressions later.
    *
    * @param bb               Blackboard
    * @param checkForDupExprs Check for duplicate expressions
@@ -528,8 +641,8 @@ public class SqlToRelConverter {
     // Then dups will be {[2, 0]}
     // and oldToNew will be {[0, 0], [1, 1], [2, 0], [3, 2]}
     RelNode rel = bb.root;
-    if (checkForDupExprs && (rel instanceof ProjectRel)) {
-      ProjectRel project = (ProjectRel) rel;
+    if (checkForDupExprs && (rel instanceof LogicalProject)) {
+      LogicalProject project = (LogicalProject) rel;
       final List<RexNode> projectExprs = project.getProjects();
       List<Integer> origins = new ArrayList<Integer>();
       int dupCount = 0;
@@ -557,12 +670,12 @@ public class SqlToRelConverter {
         }
       }
       rel =
-          new ProjectRel(
+          new LogicalProject(
               cluster,
               rel,
               Pair.left(newProjects),
               Pair.right(newProjects),
-              ProjectRel.Flags.BOXED);
+              LogicalProject.Flags.BOXED);
 
       bb.root = rel;
       distinctify(bb, false);
@@ -582,12 +695,12 @@ public class SqlToRelConverter {
       }
 
       rel =
-          new ProjectRel(
+          new LogicalProject(
               cluster,
               rel,
               Pair.left(undoProjects),
               Pair.right(undoProjects),
-              ProjectRel.Flags.BOXED);
+              LogicalProject.Flags.BOXED);
 
       bb.setRoot(
           rel,
@@ -647,7 +760,7 @@ public class SqlToRelConverter {
 
     // Create a sorter using the previously constructed collations.
     bb.setRoot(
-        new SortRel(
+        new Sort(
             cluster,
             cluster.traitSetOf(Convention.NONE, collation),
             bb.root,
@@ -667,14 +780,14 @@ public class SqlToRelConverter {
         exprs.add(rexBuilder.makeInputRef(bb.root, i));
       }
       bb.setRoot(
-          new ProjectRel(
+          new LogicalProject(
               cluster,
               cluster.traitSetOf(RelCollationImpl.PRESERVE),
               bb.root,
               exprs,
               cluster.getTypeFactory().createStructType(
                   rowType.getFieldList().subList(0, fieldCount)),
-              ProjectRelBase.Flags.BOXED),
+              Project.Flags.BOXED),
           false);
     }
   }
@@ -919,15 +1032,15 @@ public class SqlToRelConverter {
         final RelNode seek = converted.left.getInput(0); // fragile
         final int keyCount = leftKeys.size();
         final List<Integer> args = ImmutableIntList.range(0, keyCount);
-        AggregateRel aggregate =
-            new AggregateRel(cluster, seek, BitSets.of(),
+        LogicalAggregate aggregate =
+            new LogicalAggregate(cluster, seek, BitSets.of(),
                 ImmutableList.of(
                     new AggregateCall(SqlStdOperatorTable.COUNT, false,
                         ImmutableList.<Integer>of(), longType, null),
                     new AggregateCall(SqlStdOperatorTable.COUNT, false,
                         args, longType, null)));
-        JoinRel join =
-            new JoinRel(cluster, bb.root, aggregate,
+        LogicalJoin join =
+            new LogicalJoin(cluster, bb.root, aggregate,
                 rexBuilder.makeLiteral(true), JoinRelType.INNER,
                 ImmutableSet.<String>of());
         bb.setRoot(join, false);
@@ -1041,9 +1154,9 @@ public class SqlToRelConverter {
       // cross join (select count(*) as c, count(deptno) as ck from v) as ct
       // left join (select distinct deptno, true as i from v) as dt
       //   on e.deptno = dt.deptno
-      final JoinRelBase join = (JoinRelBase) root;
-      final ProjectRelBase left = (ProjectRelBase) join.getLeft();
-      final RelNode leftLeft = ((JoinRelBase) left.getInput(0)).getLeft();
+      final Join join = (Join) root;
+      final Project left = (Project) join.getLeft();
+      final RelNode leftLeft = ((Join) left.getInput(0)).getLeft();
       final int leftLeftCount = leftLeft.getRowType().getFieldCount();
       final RelDataType nullableBooleanType =
           typeFactory.createTypeWithNullability(
@@ -1234,9 +1347,10 @@ public class SqlToRelConverter {
   /**
    * Gets the list size threshold under which {@link #convertInToOr} is used.
    * Lists of this size or greater will instead be converted to use a join
-   * against an inline table ({@link ValuesRel}) rather than a predicate. A
-   * threshold of 0 forces usage of an inline table in all cases; a threshold
-   * of Integer.MAX_VALUE forces usage of OR in all cases
+   * against an inline table
+   * ({@link org.apache.calcite.rel.logical.LogicalValues}) rather than a
+   * predicate. A threshold of 0 forces usage of an inline table in all cases; a
+   * threshold of Integer.MAX_VALUE forces usage of OR in all cases
    *
    * @return threshold, default 20
    */
@@ -1305,10 +1419,10 @@ public class SqlToRelConverter {
       boolean allowLiteralsOnly,
       RelDataType targetRowType) {
     // NOTE jvs 30-Apr-2006: We combine all rows consisting entirely of
-    // literals into a single ValuesRel; this gives the optimizer a smaller
+    // literals into a single LogicalValues; this gives the optimizer a smaller
     // input tree.  For everything else (computed expressions, row
     // subqueries), we union each row in as a projection on top of a
-    // OneRowRel.
+    // LogicalOneRow.
 
     final List<List<RexLiteral>> tupleList =
         new ArrayList<List<RexLiteral>>();
@@ -1375,24 +1489,24 @@ public class SqlToRelConverter {
       }
       unionInputs.add(convertRowConstructor(bb, call));
     }
-    ValuesRel valuesRel =
-        new ValuesRel(
+    LogicalValues values =
+        new LogicalValues(
             cluster,
             rowType,
             tupleList);
     RelNode resultRel;
     if (unionInputs.isEmpty()) {
-      resultRel = valuesRel;
+      resultRel = values;
     } else {
       if (!tupleList.isEmpty()) {
-        unionInputs.add(valuesRel);
+        unionInputs.add(values);
       }
-      UnionRel unionRel =
-          new UnionRel(
+      LogicalUnion union =
+          new LogicalUnion(
               cluster,
               unionInputs,
               true);
-      resultRel = unionRel;
+      resultRel = union;
     }
     leaves.add(resultRel);
     return resultRel;
@@ -1411,7 +1525,7 @@ public class SqlToRelConverter {
     if (type.isStruct()) {
       // null literals for weird stuff like UDT's need
       // special handling during type flattening, so
-      // don't use ValuesRel for those
+      // don't use LogicalValues for those
       return null;
     }
 
@@ -1426,7 +1540,7 @@ public class SqlToRelConverter {
       assert RexLiteral.isNullLiteral(child);
 
       // NOTE jvs 22-Nov-2006:  we preserve type info
-      // in ValuesRel digest, so it's OK to lose it here
+      // in LogicalValues digest, so it's OK to lose it here
       return (RexLiteral) child;
     }
 
@@ -1724,7 +1838,7 @@ public class SqlToRelConverter {
                 tableSampleSpec.getSamplePercentage(),
                 tableSampleSpec.isRepeatable(),
                 tableSampleSpec.getRepeatableSeed());
-        bb.setRoot(new SamplingRel(cluster, bb.root, params), false);
+        bb.setRoot(new Sample(cluster, bb.root, params), false);
       } else {
         throw Util.newInternal(
             "unknown TABLESAMPLE type: " + sampleSpec);
@@ -1751,7 +1865,7 @@ public class SqlToRelConverter {
       if (shouldConvertTableAccess) {
         tableRel = toRel(table);
       } else {
-        tableRel = new TableAccessRel(cluster, table);
+        tableRel = new LogicalTableScan(cluster, table);
       }
       bb.setRoot(tableRel, true);
       if (usedDataset[0]) {
@@ -1824,15 +1938,15 @@ public class SqlToRelConverter {
       replaceSubqueries(bb, node, RelOptUtil.Logic.TRUE_FALSE_UNKNOWN);
       final RelNode childRel =
           RelOptUtil.createProject(
-              (null != bb.root) ? bb.root : new OneRowRel(cluster),
+              (null != bb.root) ? bb.root : new LogicalOneRow(cluster),
               Collections.singletonList(bb.convertExpression(node)),
               Collections.singletonList(validator.deriveAlias(node, 0)),
               true);
 
-      UncollectRel uncollectRel =
-          new UncollectRel(cluster, cluster.traitSetOf(Convention.NONE),
+      Uncollect uncollect =
+          new Uncollect(cluster, cluster.traitSetOf(Convention.NONE),
               childRel);
-      bb.setRoot(uncollectRel, true);
+      bb.setRoot(uncollect, true);
       return;
 
     case COLLECTION_TABLE:
@@ -1867,7 +1981,7 @@ public class SqlToRelConverter {
     replaceSubqueries(bb, call, RelOptUtil.Logic.TRUE_FALSE_UNKNOWN);
 
     // Expand table macro if possible. It's more efficient than
-    // TableFunctionRel.
+    // LogicalTableFunctionScan.
     if (operator instanceof SqlUserDefinedTableMacro) {
       final SqlUserDefinedTableMacro udf =
           (SqlUserDefinedTableMacro) operator;
@@ -1892,8 +2006,8 @@ public class SqlToRelConverter {
     final List<RelNode> inputs = bb.retrieveCursors();
     Set<RelColumnMapping> columnMappings =
         getColumnMappings(operator);
-    TableFunctionRel callRel =
-        new TableFunctionRel(
+    LogicalTableFunctionScan callRel =
+        new LogicalTableFunctionScan(
             cluster,
             inputs,
             rexCall,
@@ -1907,7 +2021,7 @@ public class SqlToRelConverter {
   protected void afterTableFunction(
       SqlToRelConverter.Blackboard bb,
       SqlCall call,
-      TableFunctionRel callRel) {
+      LogicalTableFunctionScan callRel) {
   }
 
   private Set<RelColumnMapping> getColumnMappings(SqlOperator op) {
@@ -2011,7 +2125,7 @@ public class SqlToRelConverter {
       }
 
       if (!correlations.isEmpty()) {
-        return new CorrelatorRel(
+        return new Correlator(
             rightRel.getCluster(),
             leftRel,
             rightRel,
@@ -2035,13 +2149,11 @@ public class SqlToRelConverter {
       leftRel = RelOptUtil.createProject(
           leftRel,
           new AbstractList<Pair<RexNode, String>>() {
-            @Override
-            public int size() {
+            @Override public int size() {
               return leftCount + extraLeftExprs.size();
             }
 
-            @Override
-            public Pair<RexNode, String> get(int index) {
+            @Override public Pair<RexNode, String> get(int index) {
               if (index < leftCount) {
                 RelDataTypeField field = fields.get(index);
                 return Pair.<RexNode, String>of(
@@ -2062,13 +2174,11 @@ public class SqlToRelConverter {
       rightRel = RelOptUtil.createProject(
           rightRel,
           new AbstractList<Pair<RexNode, String>>() {
-            @Override
-            public int size() {
+            @Override public int size() {
               return rightCount + extraRightExprs.size();
             }
 
-            @Override
-            public Pair<RexNode, String> get(int index) {
+            @Override public Pair<RexNode, String> get(int index) {
               if (index < rightCount) {
                 RelDataTypeField field = fields.get(index);
                 return Pair.<RexNode, String>of(
@@ -2616,7 +2726,7 @@ public class SqlToRelConverter {
   }
 
   /**
-   * Creates an AggregateRel.
+   * Creates an Aggregate.
    *
    * <p>In case the aggregate rel changes the order in which it projects
    * fields, the <code>groupExprProjection</code> parameter is provided, and
@@ -2630,13 +2740,13 @@ public class SqlToRelConverter {
    * @param bb       Blackboard
    * @param groupSet Bit set of ordinals of grouping columns
    * @param aggCalls Array of calls to aggregate functions
-   * @return AggregateRel
+   * @return LogicalAggregate
    */
   protected RelNode createAggregate(
       Blackboard bb,
       BitSet groupSet,
       List<AggregateCall> aggCalls) {
-    return new AggregateRel(
+    return new LogicalAggregate(
         cluster,
         bb.root,
         groupSet,
@@ -2838,7 +2948,7 @@ public class SqlToRelConverter {
     }
     switch (call.getKind()) {
     case UNION:
-      return new UnionRel(
+      return new LogicalUnion(
           cluster,
           ImmutableList.of(left, right),
           all);
@@ -2846,7 +2956,7 @@ public class SqlToRelConverter {
     case INTERSECT:
       // TODO:  all
       if (!all) {
-        return new IntersectRel(
+        return new LogicalIntersect(
             cluster,
             ImmutableList.of(left, right),
             all);
@@ -2858,7 +2968,7 @@ public class SqlToRelConverter {
     case EXCEPT:
       // TODO:  all
       if (!all) {
-        return new MinusRel(
+        return new LogicalMinus(
             cluster,
             ImmutableList.of(left, right),
             all);
@@ -2893,16 +3003,16 @@ public class SqlToRelConverter {
           targetTable,
           catalogReader,
           massagedRel,
-          TableModificationRel.Operation.INSERT,
+          LogicalTableModify.Operation.INSERT,
           null,
           false);
     }
-    return new TableModificationRel(
+    return new LogicalTableModify(
         cluster,
         targetTable,
         catalogReader,
         massagedRel,
-        TableModificationRel.Operation.INSERT,
+        LogicalTableModify.Operation.INSERT,
         null,
         false);
   }
@@ -3045,12 +3155,12 @@ public class SqlToRelConverter {
   private RelNode convertDelete(SqlDelete call) {
     RelOptTable targetTable = getTargetTable(call);
     RelNode sourceRel = convertSelect(call.getSourceSelect());
-    return new TableModificationRel(
+    return new LogicalTableModify(
         cluster,
         targetTable,
         catalogReader,
         sourceRel,
-        TableModificationRel.Operation.DELETE,
+        LogicalTableModify.Operation.DELETE,
         null,
         false);
   }
@@ -3068,12 +3178,12 @@ public class SqlToRelConverter {
 
     RelNode sourceRel = convertSelect(call.getSourceSelect());
 
-    return new TableModificationRel(
+    return new LogicalTableModify(
         cluster,
         targetTable,
         catalogReader,
         sourceRel,
-        TableModificationRel.Operation.UPDATE,
+        LogicalTableModify.Operation.UPDATE,
         targetColumnNameList,
         false);
   }
@@ -3120,17 +3230,17 @@ public class SqlToRelConverter {
       // the column; or if the expressions directly map to the source
       // table
       level1InsertExprs =
-          ((ProjectRel) insertRel.getInput(0)).getProjects();
-      if (insertRel.getInput(0).getInput(0) instanceof ProjectRel) {
+          ((LogicalProject) insertRel.getInput(0)).getProjects();
+      if (insertRel.getInput(0).getInput(0) instanceof LogicalProject) {
         level2InsertExprs =
-            ((ProjectRel) insertRel.getInput(0).getInput(0))
+            ((LogicalProject) insertRel.getInput(0).getInput(0))
                 .getProjects();
       }
       nLevel1Exprs = level1InsertExprs.size();
     }
 
-    JoinRel joinRel = (JoinRel) mergeSourceRel.getInput(0);
-    int nSourceFields = joinRel.getLeft().getRowType().getFieldCount();
+    LogicalJoin join = (LogicalJoin) mergeSourceRel.getInput(0);
+    int nSourceFields = join.getLeft().getRowType().getFieldCount();
     List<RexNode> projects = new ArrayList<RexNode>();
     for (int level1Idx = 0; level1Idx < nLevel1Exprs; level1Idx++) {
       if ((level2InsertExprs != null)
@@ -3143,20 +3253,20 @@ public class SqlToRelConverter {
       }
     }
     if (updateCall != null) {
-      final ProjectRel project = (ProjectRel) mergeSourceRel;
+      final LogicalProject project = (LogicalProject) mergeSourceRel;
       projects.addAll(
           Util.skip(project.getProjects(), nSourceFields));
     }
 
     RelNode massagedRel =
-        RelOptUtil.createProject(joinRel, projects, null, true);
+        RelOptUtil.createProject(join, projects, null, true);
 
-    return new TableModificationRel(
+    return new LogicalTableModify(
         cluster,
         targetTable,
         catalogReader,
         massagedRel,
-        TableModificationRel.Operation.MERGE,
+        LogicalTableModify.Operation.MERGE,
         targetColumnNameList,
         false);
   }
@@ -3319,13 +3429,13 @@ public class SqlToRelConverter {
         joinList.add(lastList);
       }
       lastList = new ArrayList<SqlNode>();
-      CollectRel collectRel =
-          new CollectRel(
+      Collect collect =
+          new Collect(
               cluster,
               cluster.traitSetOf(Convention.NONE),
               input,
               validator.deriveAlias(call, i));
-      joinList.add(collectRel);
+      joinList.add(collect);
     }
 
     if (joinList.size() == 0) {
@@ -3354,7 +3464,7 @@ public class SqlToRelConverter {
 
         RelNode projRel =
             RelOptUtil.createProject(
-                new OneRowRel(cluster),
+                new LogicalOneRow(cluster),
                 selectList,
                 fieldNameList);
 
@@ -3386,7 +3496,7 @@ public class SqlToRelConverter {
    * @param joinType         Join type
    * @param variablesStopped Set of names of variables which are set by the
    *                         LHS and used by the RHS and are not available to
-   *                         nodes above this JoinRel in the tree
+   *                         nodes above this LogicalJoin in the tree
    * @return A relational expression representing a join
    */
   protected RelNode createJoin(
@@ -3395,7 +3505,7 @@ public class SqlToRelConverter {
       RexNode condition,
       JoinRelType joinType,
       Set<String> variablesStopped) {
-    return new JoinRel(
+    return new LogicalJoin(
         cluster,
         left,
         right,
@@ -3532,7 +3642,7 @@ public class SqlToRelConverter {
       Blackboard bb,
       SqlCall values,
       RelDataType targetRowType) {
-    // Attempt direct conversion to ValuesRel; if that fails, deal with
+    // Attempt direct conversion to LogicalValues; if that fails, deal with
     // fancy stuff like subqueries below.
     RelNode valuesRel =
         convertRowValues(
@@ -3562,7 +3672,7 @@ public class SqlToRelConverter {
       }
       RelNode in =
           (null == tmpBb.root)
-              ? new OneRowRel(cluster)
+              ? new LogicalOneRow(cluster)
               : tmpBb.root;
       unionRels.add(
           RelOptUtil.createProject(
@@ -3580,7 +3690,7 @@ public class SqlToRelConverter {
           true);
     } else {
       bb.setRoot(
-          new UnionRel(
+          new LogicalUnion(
               cluster,
               unionRels,
               true),
@@ -3721,14 +3831,14 @@ public class SqlToRelConverter {
           leftJoinKeys.add(origLeftInputCount + leftJoinKeys.size());
         }
 
-        ProjectRel newLeftInput =
-            (ProjectRel) RelOptUtil.createProject(
+        LogicalProject newLeftInput =
+            (LogicalProject) RelOptUtil.createProject(
                 root,
                 newLeftInputExpr,
                 null,
                 true);
 
-        // maintain the group by mapping in the new ProjectRel
+        // maintain the group by mapping in the new LogicalProject
         if (mapRootRelToFieldProjection.containsKey(root)) {
           mapRootRelToFieldProjection.put(
               newLeftInput,
@@ -3808,7 +3918,7 @@ public class SqlToRelConverter {
      */
     public void setRoot(RelNode root, boolean leaf) {
       setRoot(
-          Collections.singletonList(root), root, root instanceof JoinRel);
+          Collections.singletonList(root), root, root instanceof LogicalJoin);
       if (leaf) {
         leaves.add(root);
       }
@@ -3828,8 +3938,9 @@ public class SqlToRelConverter {
     }
 
     /**
-     * Notifies this Blackboard that the root just set using {@link
-     * #setRoot(RelNode, boolean)} was derived using dataset substitution.
+     * Notifies this Blackboard that the root just set using
+     * {@link #setRoot(RelNode, boolean)} was derived using dataset
+     * substitution.
      *
      * <p>The default implementation is not interested in such
      * notifications, and does nothing.
@@ -3854,8 +3965,7 @@ public class SqlToRelConverter {
       if (nameToNodeMap != null) {
         RexNode node = nameToNodeMap.get(name);
         if (node == null) {
-          throw Util.newInternal(
-              "Unknown identifier '" + name
+          throw Util.newInternal("Unknown identifier '" + name
               + "' encountered while expanding expression" + node);
         }
         return node;
@@ -3934,8 +4044,8 @@ public class SqlToRelConverter {
               Pair.of(rel, start[0]));
           start[0] += rel.getRowType().getFieldCount();
         } else {
-          if (rel instanceof JoinRel
-              || rel instanceof AggregateRel) {
+          if (rel instanceof LogicalJoin
+              || rel instanceof LogicalAggregate) {
             start[0] += systemFieldCount;
           }
           flatten(
@@ -4192,6 +4302,7 @@ public class SqlToRelConverter {
     }
   }
 
+  /** Deferred lookup. */
   private static class DeferredLookup {
     Blackboard bb;
     String originalRelName;
@@ -4283,8 +4394,8 @@ public class SqlToRelConverter {
         new SqlNodeList(SqlParserPos.ZERO);
 
     /**
-     * Input expressions for the group columns and aggregates, in {@link
-     * RexNode} format. The first elements of the list correspond to the
+     * Input expressions for the group columns and aggregates, in
+     * {@link RexNode} format. The first elements of the list correspond to the
      * elements in {@link #groupExprs}; the remaining elements are for
      * aggregates.
      */
@@ -4440,8 +4551,8 @@ public class SqlToRelConverter {
           bb.agg = this;
         }
 
-        final Aggregation aggregation =
-            (Aggregation) call.getOperator();
+        final SqlAggFunction aggFunction =
+            (SqlAggFunction) call.getOperator();
         RelDataType type = validator.deriveType(bb.scope, call);
         boolean distinct = false;
         SqlLiteral quantifier = call.getFunctionQuantifier();
@@ -4451,7 +4562,7 @@ public class SqlToRelConverter {
         }
         final AggregateCall aggCall =
             new AggregateCall(
-                aggregation,
+                aggFunction,
                 distinct,
                 args,
                 type,
@@ -4719,9 +4830,9 @@ public class SqlToRelConverter {
      * Returns the histogram operator corresponding to a given aggregate
      * function.
      *
-     * <p>For example, <code>getHistogramOp({@link
-     * SqlStdOperatorTable#MIN}}</code> returns {@link
-     * SqlStdOperatorTable#HISTOGRAM_MIN}.
+     * <p>For example, <code>getHistogramOp
+     *({@link SqlStdOperatorTable#MIN}}</code> returns
+     * {@link SqlStdOperatorTable#HISTOGRAM_MIN}.
      *
      * @param aggFunction An aggregate function
      * @return Its histogram function, or null

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java b/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java
index 59b48d7..74fc92e 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java
@@ -14,23 +14,62 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql2rel;
-
-import java.math.*;
-import java.util.*;
-
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.rex.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.fun.*;
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.util.*;
-import org.eigenbase.util14.DateTimeUtil;
+package org.apache.calcite.sql2rel;
+
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexCallBinding;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexRangeRef;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.sql.SqlAggFunction;
+import org.apache.calcite.sql.SqlBasicCall;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlDataTypeSpec;
+import org.apache.calcite.sql.SqlFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlIntervalLiteral;
+import org.apache.calcite.sql.SqlIntervalQualifier;
+import org.apache.calcite.sql.SqlJdbcFunctionCall;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlNodeList;
+import org.apache.calcite.sql.SqlNumericLiteral;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlUtil;
+import org.apache.calcite.sql.fun.SqlArrayValueConstructor;
+import org.apache.calcite.sql.fun.SqlAvgAggFunction;
+import org.apache.calcite.sql.fun.SqlBetweenOperator;
+import org.apache.calcite.sql.fun.SqlCase;
+import org.apache.calcite.sql.fun.SqlDatetimeSubtractionOperator;
+import org.apache.calcite.sql.fun.SqlExtractFunction;
+import org.apache.calcite.sql.fun.SqlLiteralChainOperator;
+import org.apache.calcite.sql.fun.SqlMapValueConstructor;
+import org.apache.calcite.sql.fun.SqlMultisetQueryConstructor;
+import org.apache.calcite.sql.fun.SqlMultisetValueConstructor;
+import org.apache.calcite.sql.fun.SqlOverlapsOperator;
+import org.apache.calcite.sql.fun.SqlRowOperator;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.util.DateTimeUtil;
+import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
 
+import java.math.BigDecimal;
+import java.math.MathContext;
+import java.util.AbstractList;
+import java.util.ArrayList;
+import java.util.List;
+
 /**
  * Standard implementation of {@link SqlRexConvertletTable}.
  */
@@ -155,11 +194,11 @@ public class StandardConvertletTable extends ReflectiveConvertletTable {
 
     // REVIEW jvs 24-Apr-2006: This only seems to be working from within a
     // windowed agg.  I have added an optimizer rule
-    // org.eigenbase.rel.rules.ReduceAggregatesRule which handles other
-    // cases post-translation.  The reason I did that was to defer the
-    // implementation decision; e.g. we may want to push it down to a
-    // foreign server directly rather than decomposed; decomposition is
-    // easier than recognition.
+    // org.apache.calcite.rel.rules.AggregateReduceFunctionsRule which handles
+    // other cases post-translation.  The reason I did that was to defer the
+    // implementation decision; e.g. we may want to push it down to a foreign
+    // server directly rather than decomposed; decomposition is easier than
+    // recognition.
 
     // Convert "avg(<expr>)" to "cast(sum(<expr>) / count(<expr>) as
     // <type>)". We don't need to handle the empty set specially, because
@@ -623,8 +662,7 @@ public class StandardConvertletTable extends ReflectiveConvertletTable {
     if (returnType == null) {
       RexCallBinding binding =
           new RexCallBinding(cx.getTypeFactory(), fun, exprs) {
-            @Override
-            public int getGroupCount() {
+            @Override public int getGroupCount() {
               return groupCount;
             }
           };
@@ -956,6 +994,8 @@ public class StandardConvertletTable extends ReflectiveConvertletTable {
     return cx.getRexBuilder().makeCast(resType, value);
   }
 
+  /** Convertlet that handles {@code AVG} and {@code VARIANCE}
+   * windowed aggregate functions. */
   private static class AvgVarianceConvertlet implements SqlRexConvertlet {
     private final SqlAvgAggFunction.Subtype subtype;
 
@@ -1067,6 +1107,7 @@ public class StandardConvertletTable extends ReflectiveConvertletTable {
     }
   }
 
+  /** Convertlet that handles {@code FLOOR} and {@code CEIL} functions. */
   private class FloorCeilConvertlet implements SqlRexConvertlet {
     private final boolean floor;
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql2rel/SubqueryConverter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql2rel/SubqueryConverter.java b/core/src/main/java/org/apache/calcite/sql2rel/SubqueryConverter.java
index 1f3a9e3..6145619 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/SubqueryConverter.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/SubqueryConverter.java
@@ -14,10 +14,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql2rel;
+package org.apache.calcite.sql2rel;
 
-import org.eigenbase.rex.*;
-import org.eigenbase.sql.*;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.SqlCall;
 
 /**
  * SubqueryConverter provides the interface for classes that convert subqueries

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql2rel/package-info.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql2rel/package-info.java b/core/src/main/java/org/apache/calcite/sql2rel/package-info.java
index dcac14f..0b0aa01 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/package-info.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/package-info.java
@@ -18,6 +18,6 @@
 /**
  * Translates a SQL parse tree to relational expression.
  */
-package org.eigenbase.sql2rel;
+package org.apache.calcite.sql2rel;
 
 // End package-info.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/tools/FrameworkConfig.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/tools/FrameworkConfig.java b/core/src/main/java/org/apache/calcite/tools/FrameworkConfig.java
index 98d0739..d3118dd 100644
--- a/core/src/main/java/org/apache/calcite/tools/FrameworkConfig.java
+++ b/core/src/main/java/org/apache/calcite/tools/FrameworkConfig.java
@@ -14,18 +14,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.tools;
+package org.apache.calcite.tools;
 
-import net.hydromatic.optiq.SchemaPlus;
-import net.hydromatic.optiq.config.Lex;
-
-import org.eigenbase.relopt.Context;
-import org.eigenbase.relopt.RelOptCostFactory;
-import org.eigenbase.relopt.RelTraitDef;
-import org.eigenbase.reltype.RelDataTypeSystem;
-import org.eigenbase.sql.SqlOperatorTable;
-import org.eigenbase.sql.parser.SqlParserImplFactory;
-import org.eigenbase.sql2rel.SqlRexConvertletTable;
+import org.apache.calcite.config.Lex;
+import org.apache.calcite.plan.Context;
+import org.apache.calcite.plan.RelOptCostFactory;
+import org.apache.calcite.plan.RelTraitDef;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.sql.SqlOperatorTable;
+import org.apache.calcite.sql.parser.SqlParserImplFactory;
+import org.apache.calcite.sql2rel.SqlRexConvertletTable;
 
 import com.google.common.collect.ImmutableList;
 
@@ -60,7 +59,7 @@ public interface FrameworkConfig {
    *
    * <p>The common use case is when there is a single program
    * created using {@link Programs#of(RuleSet)}
-   * and {@link net.hydromatic.optiq.tools.Planner#transform}
+   * and {@link org.apache.calcite.tools.Planner#transform}
    * will only be called once.
    *
    * <p>However, consumers may also create programs
@@ -107,7 +106,7 @@ public interface FrameworkConfig {
 
   /**
    * Returns the PlannerContext that should be made available during planning by
-   * calling {@link org.eigenbase.relopt.RelOptPlanner#getContext()}.
+   * calling {@link org.apache.calcite.plan.RelOptPlanner#getContext()}.
    */
   Context getContext();
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/tools/Frameworks.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/tools/Frameworks.java b/core/src/main/java/org/apache/calcite/tools/Frameworks.java
index 7024343..4f835a5 100644
--- a/core/src/main/java/org/apache/calcite/tools/Frameworks.java
+++ b/core/src/main/java/org/apache/calcite/tools/Frameworks.java
@@ -14,29 +14,28 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.tools;
-
-import net.hydromatic.optiq.SchemaPlus;
-import net.hydromatic.optiq.config.Lex;
-import net.hydromatic.optiq.config.OptiqConnectionProperty;
-import net.hydromatic.optiq.jdbc.OptiqConnection;
-import net.hydromatic.optiq.jdbc.OptiqSchema;
-import net.hydromatic.optiq.prepare.OptiqPrepareImpl;
-import net.hydromatic.optiq.prepare.PlannerImpl;
-import net.hydromatic.optiq.server.OptiqServerStatement;
-
-import org.eigenbase.relopt.Context;
-import org.eigenbase.relopt.RelOptCluster;
-import org.eigenbase.relopt.RelOptCostFactory;
-import org.eigenbase.relopt.RelOptSchema;
-import org.eigenbase.relopt.RelTraitDef;
-import org.eigenbase.reltype.RelDataTypeSystem;
-import org.eigenbase.sql.SqlOperatorTable;
-import org.eigenbase.sql.fun.SqlStdOperatorTable;
-import org.eigenbase.sql.parser.SqlParserImplFactory;
-import org.eigenbase.sql.parser.impl.SqlParserImpl;
-import org.eigenbase.sql2rel.SqlRexConvertletTable;
-import org.eigenbase.sql2rel.StandardConvertletTable;
+package org.apache.calcite.tools;
+
+import org.apache.calcite.config.CalciteConnectionProperty;
+import org.apache.calcite.config.Lex;
+import org.apache.calcite.jdbc.CalciteConnection;
+import org.apache.calcite.jdbc.CalciteSchema;
+import org.apache.calcite.plan.Context;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCostFactory;
+import org.apache.calcite.plan.RelOptSchema;
+import org.apache.calcite.plan.RelTraitDef;
+import org.apache.calcite.prepare.CalcitePrepareImpl;
+import org.apache.calcite.prepare.PlannerImpl;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.server.CalciteServerStatement;
+import org.apache.calcite.sql.SqlOperatorTable;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.parser.SqlParserImplFactory;
+import org.apache.calcite.sql.parser.impl.SqlParserImpl;
+import org.apache.calcite.sql2rel.SqlRexConvertletTable;
+import org.apache.calcite.sql2rel.StandardConvertletTable;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
@@ -75,7 +74,7 @@ public class Frameworks {
   /** Piece of code to be run in a context where a planner and statement are
    * available. The planner is accessible from the {@code cluster} parameter, as
    * are several other useful objects. The connection and
-   * {@link net.hydromatic.optiq.DataContext} are accessible from the
+   * {@link org.apache.calcite.DataContext} are accessible from the
    * statement. */
   public abstract static class PrepareAction<R> {
     private final FrameworkConfig config;
@@ -94,7 +93,7 @@ public class Frameworks {
     }
 
     public abstract R apply(RelOptCluster cluster, RelOptSchema relOptSchema,
-        SchemaPlus rootSchema, OptiqServerStatement statement);
+        SchemaPlus rootSchema, CalciteServerStatement statement);
   }
 
   /**
@@ -109,7 +108,7 @@ public class Frameworks {
     return withPrepare(
         new Frameworks.PrepareAction<R>(config) {
           public R apply(RelOptCluster cluster, RelOptSchema relOptSchema,
-              SchemaPlus rootSchema, OptiqServerStatement statement) {
+              SchemaPlus rootSchema, CalciteServerStatement statement) {
             return action.apply(cluster, relOptSchema, rootSchema);
           }
         });
@@ -136,19 +135,19 @@ public class Frameworks {
    */
   public static <R> R withPrepare(PrepareAction<R> action) {
     try {
-      Class.forName("net.hydromatic.optiq.jdbc.Driver");
       final Properties info = new Properties();
       if (action.config.getTypeSystem() != RelDataTypeSystem.DEFAULT) {
-        info.setProperty(OptiqConnectionProperty.TYPE_SYSTEM.camelName(),
+        info.setProperty(CalciteConnectionProperty.TYPE_SYSTEM.camelName(),
             action.config.getTypeSystem().getClass().getName());
       }
       Connection connection =
           DriverManager.getConnection("jdbc:calcite:", info);
-      OptiqConnection optiqConnection =
-          connection.unwrap(OptiqConnection.class);
-      final OptiqServerStatement statement =
-          optiqConnection.createStatement().unwrap(OptiqServerStatement.class);
-      return new OptiqPrepareImpl().perform(statement, action);
+      CalciteConnection calciteConnection =
+          connection.unwrap(CalciteConnection.class);
+      final CalciteServerStatement statement =
+          calciteConnection.createStatement()
+              .unwrap(CalciteServerStatement.class);
+      return new CalcitePrepareImpl().perform(statement, action);
     } catch (Exception e) {
       throw new RuntimeException(e);
     }
@@ -161,7 +160,7 @@ public class Frameworks {
    *    definitions of tables, columns etc.
    */
   public static SchemaPlus createRootSchema(boolean addMetadataSchema) {
-    return OptiqSchema.createRootSchema(addMetadataSchema).plus();
+    return CalciteSchema.createRootSchema(addMetadataSchema).plus();
   }
 
   public static ConfigBuilder newConfigBuilder() {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/tools/Planner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/tools/Planner.java b/core/src/main/java/org/apache/calcite/tools/Planner.java
index 5b7e776..98c1649 100644
--- a/core/src/main/java/org/apache/calcite/tools/Planner.java
+++ b/core/src/main/java/org/apache/calcite/tools/Planner.java
@@ -14,13 +14,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.tools;
+package org.apache.calcite.tools;
 
-import org.eigenbase.rel.RelNode;
-import org.eigenbase.relopt.RelTraitSet;
-import org.eigenbase.reltype.RelDataTypeFactory;
-import org.eigenbase.sql.SqlNode;
-import org.eigenbase.sql.parser.SqlParseException;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.parser.SqlParseException;
 
 /**
  * A fa&ccedil;ade that covers Calcite's query planning process: parse SQL,
@@ -38,7 +38,7 @@ public interface Planner {
    *
    * @param sql The SQL statement to parse.
    * @return The root node of the SQL parse tree.
-   * @throws org.eigenbase.sql.parser.SqlParseException on parse error
+   * @throws org.apache.calcite.sql.parser.SqlParseException on parse error
    */
   SqlNode parse(String sql) throws SqlParseException;
 
@@ -54,11 +54,11 @@ public interface Planner {
   /**
    * Converts a SQL parse tree into a tree of relational expressions.
    *
-   * <p>You must call {@link #validate(org.eigenbase.sql.SqlNode)} first.</p>
+   * <p>You must call {@link #validate(org.apache.calcite.sql.SqlNode)} first.
    *
    * @param sql The root node of the SQL parse tree.
    * @return The root node of the newly generated RelNode tree.
-   * @throws net.hydromatic.optiq.tools.RelConversionException if the node
+   * @throws org.apache.calcite.tools.RelConversionException if the node
    * cannot be converted or has not been validated
    */
   RelNode convert(SqlNode sql) throws RelConversionException;
@@ -78,7 +78,7 @@ public interface Planner {
    *                             at the termination of the planning cycle.
    * @param rel The root of the RelNode tree to convert.
    * @return The root of the new RelNode tree.
-   * @throws net.hydromatic.optiq.tools.RelConversionException on conversion
+   * @throws org.apache.calcite.tools.RelConversionException on conversion
    *     error
    */
   RelNode transform(int ruleSetIndex,

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/tools/Program.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/tools/Program.java b/core/src/main/java/org/apache/calcite/tools/Program.java
index 04898e0..1bfa4cf 100644
--- a/core/src/main/java/org/apache/calcite/tools/Program.java
+++ b/core/src/main/java/org/apache/calcite/tools/Program.java
@@ -14,11 +14,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.tools;
+package org.apache.calcite.tools;
 
-import org.eigenbase.rel.RelNode;
-import org.eigenbase.relopt.RelOptPlanner;
-import org.eigenbase.relopt.RelTraitSet;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
 
 /**
  * Program that transforms a relational expression into another relational
@@ -27,7 +27,7 @@ import org.eigenbase.relopt.RelTraitSet;
  * <p>A planner is a sequence of programs, each of which is sometimes called
  * a "phase".
  * The most typical program is an invocation of the volcano planner with a
- * particular {@link net.hydromatic.optiq.tools.RuleSet}.</p>
+ * particular {@link org.apache.calcite.tools.RuleSet}.</p>
  */
 public interface Program {
   RelNode run(RelOptPlanner planner, RelNode rel,

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/tools/Programs.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/tools/Programs.java b/core/src/main/java/org/apache/calcite/tools/Programs.java
index 1364467..49c77ae 100644
--- a/core/src/main/java/org/apache/calcite/tools/Programs.java
+++ b/core/src/main/java/org/apache/calcite/tools/Programs.java
@@ -14,25 +14,54 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.tools;
-
-import net.hydromatic.optiq.prepare.OptiqPrepareImpl;
-import net.hydromatic.optiq.rules.java.JavaRules;
-
-import org.eigenbase.rel.RelNode;
-import org.eigenbase.rel.metadata.ChainedRelMetadataProvider;
-import org.eigenbase.rel.metadata.DefaultRelMetadataProvider;
-import org.eigenbase.rel.metadata.RelMetadataProvider;
-import org.eigenbase.rel.rules.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.relopt.hep.*;
+package org.apache.calcite.tools;
+
+import org.apache.calcite.adapter.enumerable.EnumerableRules;
+import org.apache.calcite.plan.RelOptCostImpl;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.plan.hep.HepMatchOrder;
+import org.apache.calcite.plan.hep.HepPlanner;
+import org.apache.calcite.plan.hep.HepProgram;
+import org.apache.calcite.plan.hep.HepProgramBuilder;
+import org.apache.calcite.prepare.CalcitePrepareImpl;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.metadata.ChainedRelMetadataProvider;
+import org.apache.calcite.rel.metadata.DefaultRelMetadataProvider;
+import org.apache.calcite.rel.metadata.RelMetadataProvider;
+import org.apache.calcite.rel.rules.AggregateExpandDistinctAggregatesRule;
+import org.apache.calcite.rel.rules.AggregateReduceFunctionsRule;
+import org.apache.calcite.rel.rules.AggregateStarTableRule;
+import org.apache.calcite.rel.rules.CalcMergeRule;
+import org.apache.calcite.rel.rules.FilterAggregateTransposeRule;
+import org.apache.calcite.rel.rules.FilterCalcMergeRule;
+import org.apache.calcite.rel.rules.FilterJoinRule;
+import org.apache.calcite.rel.rules.FilterProjectTransposeRule;
+import org.apache.calcite.rel.rules.FilterTableRule;
+import org.apache.calcite.rel.rules.FilterToCalcRule;
+import org.apache.calcite.rel.rules.JoinAssociateRule;
+import org.apache.calcite.rel.rules.JoinCommuteRule;
+import org.apache.calcite.rel.rules.JoinPushThroughJoinRule;
+import org.apache.calcite.rel.rules.JoinToMultiJoinRule;
+import org.apache.calcite.rel.rules.LoptOptimizeJoinRule;
+import org.apache.calcite.rel.rules.MultiJoinOptimizeBushyRule;
+import org.apache.calcite.rel.rules.ProjectCalcMergeRule;
+import org.apache.calcite.rel.rules.ProjectMergeRule;
+import org.apache.calcite.rel.rules.ProjectToCalcRule;
+import org.apache.calcite.rel.rules.SemiJoinRule;
+import org.apache.calcite.rel.rules.SortProjectTransposeRule;
+import org.apache.calcite.rel.rules.TableScanRule;
 
 import com.google.common.base.Function;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Lists;
 
-import java.util.*;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
 
 /**
  * Utilities for creating {@link Program}s.
@@ -47,21 +76,21 @@ public class Programs {
 
   public static final ImmutableList<RelOptRule> CALC_RULES =
       ImmutableList.of(
-          JavaRules.ENUMERABLE_CALC_RULE,
-          JavaRules.ENUMERABLE_FILTER_TO_CALC_RULE,
-          JavaRules.ENUMERABLE_PROJECT_TO_CALC_RULE,
-          MergeCalcRule.INSTANCE,
-          MergeFilterOntoCalcRule.INSTANCE,
-          MergeProjectOntoCalcRule.INSTANCE,
+          EnumerableRules.ENUMERABLE_CALC_RULE,
+          EnumerableRules.ENUMERABLE_FILTER_TO_CALC_RULE,
+          EnumerableRules.ENUMERABLE_PROJECT_TO_CALC_RULE,
+          CalcMergeRule.INSTANCE,
+          FilterCalcMergeRule.INSTANCE,
+          ProjectCalcMergeRule.INSTANCE,
           FilterToCalcRule.INSTANCE,
           ProjectToCalcRule.INSTANCE,
-          MergeCalcRule.INSTANCE,
+          CalcMergeRule.INSTANCE,
 
           // REVIEW jvs 9-Apr-2006: Do we still need these two?  Doesn't the
-          // combination of MergeCalcRule, FilterToCalcRule, and
+          // combination of CalcMergeRule, FilterToCalcRule, and
           // ProjectToCalcRule have the same effect?
-          MergeFilterOntoCalcRule.INSTANCE,
-          MergeProjectOntoCalcRule.INSTANCE);
+          FilterCalcMergeRule.INSTANCE,
+          ProjectCalcMergeRule.INSTANCE);
 
   /** Program that converts filters and projects to calcs. */
   public static final Program CALC_PROGRAM =
@@ -69,38 +98,38 @@ public class Programs {
 
   public static final ImmutableSet<RelOptRule> RULE_SET =
       ImmutableSet.of(
-          JavaRules.ENUMERABLE_JOIN_RULE,
-          JavaRules.ENUMERABLE_SEMI_JOIN_RULE,
-          JavaRules.ENUMERABLE_PROJECT_RULE,
-          JavaRules.ENUMERABLE_FILTER_RULE,
-          JavaRules.ENUMERABLE_AGGREGATE_RULE,
-          JavaRules.ENUMERABLE_SORT_RULE,
-          JavaRules.ENUMERABLE_LIMIT_RULE,
-          JavaRules.ENUMERABLE_UNION_RULE,
-          JavaRules.ENUMERABLE_INTERSECT_RULE,
-          JavaRules.ENUMERABLE_MINUS_RULE,
-          JavaRules.ENUMERABLE_TABLE_MODIFICATION_RULE,
-          JavaRules.ENUMERABLE_VALUES_RULE,
-          JavaRules.ENUMERABLE_WINDOW_RULE,
-          JavaRules.ENUMERABLE_ONE_ROW_RULE,
-          JavaRules.ENUMERABLE_EMPTY_RULE,
+          EnumerableRules.ENUMERABLE_JOIN_RULE,
+          EnumerableRules.ENUMERABLE_SEMI_JOIN_RULE,
+          EnumerableRules.ENUMERABLE_PROJECT_RULE,
+          EnumerableRules.ENUMERABLE_FILTER_RULE,
+          EnumerableRules.ENUMERABLE_AGGREGATE_RULE,
+          EnumerableRules.ENUMERABLE_SORT_RULE,
+          EnumerableRules.ENUMERABLE_LIMIT_RULE,
+          EnumerableRules.ENUMERABLE_UNION_RULE,
+          EnumerableRules.ENUMERABLE_INTERSECT_RULE,
+          EnumerableRules.ENUMERABLE_MINUS_RULE,
+          EnumerableRules.ENUMERABLE_TABLE_MODIFICATION_RULE,
+          EnumerableRules.ENUMERABLE_VALUES_RULE,
+          EnumerableRules.ENUMERABLE_WINDOW_RULE,
+          EnumerableRules.ENUMERABLE_ONE_ROW_RULE,
+          EnumerableRules.ENUMERABLE_EMPTY_RULE,
           SemiJoinRule.INSTANCE,
-          TableAccessRule.INSTANCE,
-          OptiqPrepareImpl.COMMUTE
-              ? CommutativeJoinRule.INSTANCE
-              : MergeProjectRule.INSTANCE,
+          TableScanRule.INSTANCE,
+          CalcitePrepareImpl.COMMUTE
+              ? JoinAssociateRule.INSTANCE
+              : ProjectMergeRule.INSTANCE,
           AggregateStarTableRule.INSTANCE,
           AggregateStarTableRule.INSTANCE2,
           FilterTableRule.INSTANCE,
-          PushFilterPastProjectRule.INSTANCE,
-          PushFilterPastJoinRule.FILTER_ON_JOIN,
-          RemoveDistinctAggregateRule.INSTANCE,
-          ReduceAggregatesRule.INSTANCE,
+          FilterProjectTransposeRule.INSTANCE,
+          FilterJoinRule.FILTER_ON_JOIN,
+          AggregateExpandDistinctAggregatesRule.INSTANCE,
+          AggregateReduceFunctionsRule.INSTANCE,
           FilterAggregateTransposeRule.INSTANCE,
-          SwapJoinRule.INSTANCE,
-          PushJoinThroughJoinRule.RIGHT,
-          PushJoinThroughJoinRule.LEFT,
-          PushSortPastProjectRule.INSTANCE);
+          JoinCommuteRule.INSTANCE,
+          JoinPushThroughJoinRule.RIGHT,
+          JoinPushThroughJoinRule.LEFT,
+          SortProjectTransposeRule.INSTANCE);
 
   // private constructor for utility class
   private Programs() {}
@@ -170,9 +199,9 @@ public class Programs {
   }
 
   /** Creates a program that invokes heuristic join-order optimization
-   * (via {@link org.eigenbase.rel.rules.ConvertMultiJoinRule},
-   * {@link org.eigenbase.rel.rules.MultiJoinRel} and
-   * {@link org.eigenbase.rel.rules.LoptOptimizeJoinRule})
+   * (via {@link org.apache.calcite.rel.rules.JoinToMultiJoinRule},
+   * {@link org.apache.calcite.rel.rules.MultiJoin} and
+   * {@link org.apache.calcite.rel.rules.LoptOptimizeJoinRule})
    * if there are 6 or more joins (7 or more relations). */
   public static Program heuristicJoinOrder(final Collection<RelOptRule> rules,
       final boolean bushy, final int minJoinCount) {
@@ -184,27 +213,27 @@ public class Programs {
         if (joinCount < minJoinCount) {
           program = ofRules(rules);
         } else {
-          // Create a program that gathers together joins as a MultiJoinRel.
+          // Create a program that gathers together joins as a MultiJoin.
           final HepProgram hep = new HepProgramBuilder()
-              .addRuleInstance(PushFilterPastJoinRule.FILTER_ON_JOIN)
+              .addRuleInstance(FilterJoinRule.FILTER_ON_JOIN)
               .addMatchOrder(HepMatchOrder.BOTTOM_UP)
-              .addRuleInstance(ConvertMultiJoinRule.INSTANCE)
+              .addRuleInstance(JoinToMultiJoinRule.INSTANCE)
               .build();
           final Program program1 =
               of(hep, false, new DefaultRelMetadataProvider());
 
-          // Create a program that contains a rule to expand a MultiJoinRel
+          // Create a program that contains a rule to expand a MultiJoin
           // into heuristically ordered joins.
-          // We use the rule set passed in, but remove SwapJoinRule and
-          // PushJoinThroughJoinRule, because they cause exhaustive search.
+          // We use the rule set passed in, but remove JoinCommuteRule and
+          // JoinPushThroughJoinRule, because they cause exhaustive search.
           final List<RelOptRule> list = Lists.newArrayList(rules);
           list.removeAll(
-              ImmutableList.of(SwapJoinRule.INSTANCE,
-                  CommutativeJoinRule.INSTANCE,
-                  PushJoinThroughJoinRule.LEFT,
-                  PushJoinThroughJoinRule.RIGHT));
+              ImmutableList.of(JoinCommuteRule.INSTANCE,
+                  JoinAssociateRule.INSTANCE,
+                  JoinPushThroughJoinRule.LEFT,
+                  JoinPushThroughJoinRule.RIGHT));
           list.add(bushy
-              ? OptimizeBushyJoinRule.INSTANCE
+              ? MultiJoinOptimizeBushyRule.INSTANCE
               : LoptOptimizeJoinRule.INSTANCE);
           final Program program2 = ofRules(list);
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/tools/RelConversionException.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/tools/RelConversionException.java b/core/src/main/java/org/apache/calcite/tools/RelConversionException.java
index c03aaef..6c1cecc 100644
--- a/core/src/main/java/org/apache/calcite/tools/RelConversionException.java
+++ b/core/src/main/java/org/apache/calcite/tools/RelConversionException.java
@@ -14,11 +14,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.tools;
+package org.apache.calcite.tools;
 
 /**
  * An Exception thrown when attempting conversion to a set of
- * {@link org.eigenbase.rel.RelNode}s.
+ * {@link org.apache.calcite.rel.RelNode}s.
  */
 public class RelConversionException extends Exception {
   /** Creates a RelConversionException with the specified detail message and

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/tools/RuleSet.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/tools/RuleSet.java b/core/src/main/java/org/apache/calcite/tools/RuleSet.java
index 8c467b1..b9b92ea 100644
--- a/core/src/main/java/org/apache/calcite/tools/RuleSet.java
+++ b/core/src/main/java/org/apache/calcite/tools/RuleSet.java
@@ -14,9 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.tools;
+package org.apache.calcite.tools;
 
-import org.eigenbase.relopt.RelOptRule;
+import org.apache.calcite.plan.RelOptRule;
 
 /**
  * A set rules associated with a particular

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/tools/RuleSets.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/tools/RuleSets.java b/core/src/main/java/org/apache/calcite/tools/RuleSets.java
index 7d9857b..8a1cdc0 100644
--- a/core/src/main/java/org/apache/calcite/tools/RuleSets.java
+++ b/core/src/main/java/org/apache/calcite/tools/RuleSets.java
@@ -14,9 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.tools;
+package org.apache.calcite.tools;
 
-import org.eigenbase.relopt.RelOptRule;
+import org.apache.calcite.plan.RelOptRule;
 
 import com.google.common.collect.ImmutableList;
 
@@ -26,7 +26,7 @@ import java.util.Iterator;
 /**
  * Utilities for creating and composing rule sets.
  *
- * @see net.hydromatic.optiq.tools.RuleSet
+ * @see org.apache.calcite.tools.RuleSet
  */
 public class RuleSets {
   private RuleSets() {
@@ -50,13 +50,11 @@ public class RuleSets {
       this.rules = rules;
     }
 
-    @Override
-    public int hashCode() {
+    @Override public int hashCode() {
       return rules.hashCode();
     }
 
-    @Override
-    public boolean equals(Object obj) {
+    @Override public boolean equals(Object obj) {
       return obj == this
           || obj instanceof ListRuleSet
           && rules.equals(((ListRuleSet) obj).rules);

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/tools/ValidationException.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/tools/ValidationException.java b/core/src/main/java/org/apache/calcite/tools/ValidationException.java
index 4046f9e..9651c71 100644
--- a/core/src/main/java/org/apache/calcite/tools/ValidationException.java
+++ b/core/src/main/java/org/apache/calcite/tools/ValidationException.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.tools;
+package org.apache.calcite.tools;
 
 /**
  * An Exception thrown when attempting to validate a SQL parse tree.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/tools/package-info.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/tools/package-info.java b/core/src/main/java/org/apache/calcite/tools/package-info.java
index 166259a..3273ae2 100644
--- a/core/src/main/java/org/apache/calcite/tools/package-info.java
+++ b/core/src/main/java/org/apache/calcite/tools/package-info.java
@@ -18,6 +18,6 @@
 /**
  * Provides utility classes.
  */
-package net.hydromatic.optiq.tools;
+package org.apache.calcite.tools;
 
 // End package-info.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/BarfingInvocationHandler.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/BarfingInvocationHandler.java b/core/src/main/java/org/apache/calcite/util/BarfingInvocationHandler.java
index d83b057..94c50a6 100644
--- a/core/src/main/java/org/apache/calcite/util/BarfingInvocationHandler.java
+++ b/core/src/main/java/org/apache/calcite/util/BarfingInvocationHandler.java
@@ -14,9 +14,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.util;
+package org.apache.calcite.util;
 
-import java.lang.reflect.*;
+import java.lang.reflect.InvocationHandler;
+import java.lang.reflect.Method;
+import java.lang.reflect.UndeclaredThrowableException;
 
 /**
  * A class derived from <code>BarfingInvocationHandler</code> handles a method

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/BasicDatetime.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/BasicDatetime.java b/core/src/main/java/org/apache/calcite/util/BasicDatetime.java
index f0f48ff..0d38bc0 100644
--- a/core/src/main/java/org/apache/calcite/util/BasicDatetime.java
+++ b/core/src/main/java/org/apache/calcite/util/BasicDatetime.java
@@ -14,9 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.util14;
+package org.apache.calcite.util;
 
-import java.util.*;
+import java.util.TimeZone;
 
 /**
  * BasicDatetime is an interface for dates, times, or timestamps that can be
@@ -35,9 +35,9 @@ import java.util.*;
  *
  * <p>A zoned time represents a time that was created in a particular time zone.
  * It may contain date and/or time components that are valid when interpreted
- * relative to a specified time zone, according to a {@link java.util.Calendar
- * Calendar}. Jdbc types, such as {@link java.sql.Date} typically contain zoned
- * times.
+ * relative to a specified time zone, according to a
+ * {@link java.util.Calendar Calendar}. Jdbc types, such as
+ * {@link java.sql.Date} typically contain zoned times.
  */
 public interface BasicDatetime {
   //~ Methods ----------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/Benchmark.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/Benchmark.java b/core/src/main/java/org/apache/calcite/util/Benchmark.java
index 95aaa6c..b992427 100644
--- a/core/src/main/java/org/apache/calcite/util/Benchmark.java
+++ b/core/src/main/java/org/apache/calcite/util/Benchmark.java
@@ -14,15 +14,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.util;
+package org.apache.calcite.util;
+
+import org.apache.calcite.linq4j.function.Function1;
 
 import java.text.MessageFormat;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
-import net.hydromatic.linq4j.function.Function1;
-
 /**
  * Helps to run benchmarks by running the same task repeatedly and averaging
  * the running times.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/BitSets.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/BitSets.java b/core/src/main/java/org/apache/calcite/util/BitSets.java
index 0616fbb..0d79c85 100644
--- a/core/src/main/java/org/apache/calcite/util/BitSets.java
+++ b/core/src/main/java/org/apache/calcite/util/BitSets.java
@@ -14,12 +14,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.util;
+package org.apache.calcite.util;
 
-import org.eigenbase.util.ImmutableIntList;
-import org.eigenbase.util.IntList;
-
-import java.util.*;
+import java.util.BitSet;
+import java.util.Iterator;
+import java.util.SortedMap;
+import java.util.TreeMap;
 
 /**
  * Utility functions for {@link BitSet}.
@@ -265,7 +265,7 @@ public final class BitSets {
   }
 
   /** Populates a {@link BitSet} from an
-   *  {@link org.eigenbase.util.ImmutableIntList}. */
+   *  {@link ImmutableIntList}. */
   public static void populate(BitSet bitSet, ImmutableIntList list) {
     for (int i = 0; i < list.size(); i++) {
       bitSet.set(list.getInt(i));

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/BitString.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/BitString.java b/core/src/main/java/org/apache/calcite/util/BitString.java
index 1dc6a95..ab7e9c3 100644
--- a/core/src/main/java/org/apache/calcite/util/BitString.java
+++ b/core/src/main/java/org/apache/calcite/util/BitString.java
@@ -14,13 +14,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.util;
+package org.apache.calcite.util;
 
-import java.math.*;
+import java.math.BigInteger;
 import java.util.List;
 
-import org.eigenbase.util14.*;
-
 /**
  * String of bits.
  *
@@ -28,12 +26,13 @@ import org.eigenbase.util14.*;
  * specified length. The length is preserved even if this means that the bit
  * string has leading '0's.
  *
- * <p>You can create a bit string from a string of 0s and 1s ({@link
- * #BitString(String, int)} or {@link #createFromBitString}), or from a string of
- * hex digits ({@link #createFromHexString}). You can convert it to a byte array
- * ({@link #getAsByteArray}), to a bit string ({@link #toBitString}), or to a
- * hex string ({@link #toHexString}). A utility method {@link
- * #toByteArrayFromBitString} converts a bit string directly to a byte array.
+ * <p>You can create a bit string from a string of 0s and 1s
+ * ({@link #BitString(String, int)} or {@link #createFromBitString}), or from a
+ * string of hex digits ({@link #createFromHexString}). You can convert it to a
+ * byte array ({@link #getAsByteArray}), to a bit string ({@link #toBitString}),
+ * or to a hex string ({@link #toHexString}). A utility method
+ * {@link #toByteArrayFromBitString} converts a bit string directly to a byte
+ * array.
  *
  * <p>This class is immutable: once created, none of the methods modify the
  * value.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/Bug.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/Bug.java b/core/src/main/java/org/apache/calcite/util/Bug.java
index 37a9db6..b851f27 100644
--- a/core/src/main/java/org/apache/calcite/util/Bug.java
+++ b/core/src/main/java/org/apache/calcite/util/Bug.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.util;
+package org.apache.calcite.util;
 
 /**
  * Holder for a list of constants describing which bugs which have not been
@@ -29,7 +29,7 @@ package org.eigenbase.util;
  * <pre>Statement stmt = connection.createStatement();
  * int rowCount = stmt.execute(
  *     "INSERT INTO FemaleEmps SELECT * FROM Emps WHERE gender = 'F'");
- * if (Bug.OPTIQ123_FIXED) {
+ * if (Bug.CALCITE_123_FIXED) {
  *    assertEquals(rowCount, 5);
  * }</pre>
  * </blockquote>

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java b/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
index 9ed9b8f..8ad1742 100644
--- a/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
+++ b/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
@@ -14,22 +14,47 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq;
+package org.apache.calcite.util;
 
-import net.hydromatic.linq4j.*;
-import net.hydromatic.linq4j.expressions.FunctionExpression;
-import net.hydromatic.linq4j.expressions.Primitive;
-import net.hydromatic.linq4j.expressions.Types;
-import net.hydromatic.linq4j.function.*;
-
-import net.hydromatic.optiq.impl.interpreter.Row;
-import net.hydromatic.optiq.impl.java.ReflectiveSchema;
-import net.hydromatic.optiq.impl.jdbc.JdbcSchema;
-import net.hydromatic.optiq.runtime.*;
-
-import org.eigenbase.rel.metadata.Metadata;
-import org.eigenbase.rex.RexNode;
-import org.eigenbase.sql.SqlExplainLevel;
+import org.apache.calcite.DataContext;
+import org.apache.calcite.adapter.java.ReflectiveSchema;
+import org.apache.calcite.adapter.jdbc.JdbcSchema;
+import org.apache.calcite.interpreter.Row;
+import org.apache.calcite.linq4j.AbstractEnumerable;
+import org.apache.calcite.linq4j.Enumerable;
+import org.apache.calcite.linq4j.Enumerator;
+import org.apache.calcite.linq4j.ExtendedEnumerable;
+import org.apache.calcite.linq4j.Linq4j;
+import org.apache.calcite.linq4j.QueryProvider;
+import org.apache.calcite.linq4j.Queryable;
+import org.apache.calcite.linq4j.function.EqualityComparer;
+import org.apache.calcite.linq4j.function.Function0;
+import org.apache.calcite.linq4j.function.Function1;
+import org.apache.calcite.linq4j.function.Function2;
+import org.apache.calcite.linq4j.function.Functions;
+import org.apache.calcite.linq4j.function.Predicate1;
+import org.apache.calcite.linq4j.function.Predicate2;
+import org.apache.calcite.linq4j.tree.FunctionExpression;
+import org.apache.calcite.linq4j.tree.Primitive;
+import org.apache.calcite.linq4j.tree.Types;
+import org.apache.calcite.rel.metadata.Metadata;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.runtime.BinarySearch;
+import org.apache.calcite.runtime.Bindable;
+import org.apache.calcite.runtime.Enumerables;
+import org.apache.calcite.runtime.FlatLists;
+import org.apache.calcite.runtime.ResultSetEnumerable;
+import org.apache.calcite.runtime.SortedMultiMap;
+import org.apache.calcite.runtime.SqlFunctions;
+import org.apache.calcite.runtime.Typed;
+import org.apache.calcite.schema.FilterableTable;
+import org.apache.calcite.schema.ModifiableTable;
+import org.apache.calcite.schema.QueryableTable;
+import org.apache.calcite.schema.ScannableTable;
+import org.apache.calcite.schema.Schema;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.schema.Schemas;
+import org.apache.calcite.sql.SqlExplainLevel;
 
 import com.google.common.collect.ImmutableMap;
 
@@ -38,15 +63,34 @@ import java.lang.reflect.Method;
 import java.sql.ResultSet;
 import java.sql.Time;
 import java.sql.Timestamp;
-import java.util.*;
+import java.util.BitSet;
+import java.util.Calendar;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.TimeZone;
 import javax.sql.DataSource;
 
-import static org.eigenbase.rel.metadata.BuiltInMetadata.*;
+import static org.apache.calcite.rel.metadata.BuiltInMetadata.ColumnOrigin;
+import static org.apache.calcite.rel.metadata.BuiltInMetadata.ColumnUniqueness;
+import static org.apache.calcite.rel.metadata.BuiltInMetadata.CumulativeCost;
+import static org.apache.calcite.rel.metadata.BuiltInMetadata.DistinctRowCount;
+import static org.apache.calcite.rel.metadata.BuiltInMetadata.ExplainVisibility;
+import static org.apache.calcite.rel.metadata.BuiltInMetadata.NonCumulativeCost;
+import static org.apache.calcite.rel.metadata.BuiltInMetadata.PercentageOriginalRows;
+import static org.apache.calcite.rel.metadata.BuiltInMetadata.PopulationSize;
+import static org.apache.calcite.rel.metadata.BuiltInMetadata.Predicates;
+import static org.apache.calcite.rel.metadata.BuiltInMetadata.RowCount;
+import static org.apache.calcite.rel.metadata.BuiltInMetadata.Selectivity;
+import static org.apache.calcite.rel.metadata.BuiltInMetadata.UniqueKeys;
 
 /**
- * Builtin methods.
+ * Built-in methods.
  */
-public enum BuiltinMethod {
+public enum BuiltInMethod {
   QUERYABLE_SELECT(Queryable.class, "select", FunctionExpression.class),
   QUERYABLE_AS_ENUMERABLE(Queryable.class, "asEnumerable"),
   QUERYABLE_TABLE_AS_QUERYABLE(QueryableTable.class, "asQueryable",
@@ -241,12 +285,12 @@ public enum BuiltinMethod {
   public final Method method;
   public final Constructor constructor;
 
-  public static final ImmutableMap<Method, BuiltinMethod> MAP;
+  public static final ImmutableMap<Method, BuiltInMethod> MAP;
 
   static {
-    final ImmutableMap.Builder<Method, BuiltinMethod> builder =
+    final ImmutableMap.Builder<Method, BuiltInMethod> builder =
         ImmutableMap.builder();
-    for (BuiltinMethod value : BuiltinMethod.values()) {
+    for (BuiltInMethod value : BuiltInMethod.values()) {
       if (value.method != null) {
         builder.put(value.method, value);
       }
@@ -254,15 +298,15 @@ public enum BuiltinMethod {
     MAP = builder.build();
   }
 
-  BuiltinMethod(Class clazz, String methodName, Class... argumentTypes) {
+  BuiltInMethod(Class clazz, String methodName, Class... argumentTypes) {
     this.method = Types.lookupMethod(clazz, methodName, argumentTypes);
     this.constructor = null;
   }
 
-  BuiltinMethod(Class clazz, Class... argumentTypes) {
+  BuiltInMethod(Class clazz, Class... argumentTypes) {
     this.method = null;
     this.constructor = Types.lookupConstructor(clazz, argumentTypes);
   }
 }
 
-// End BuiltinMethod.java
+// End BuiltInMethod.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/CalciteParserException.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/CalciteParserException.java b/core/src/main/java/org/apache/calcite/util/CalciteParserException.java
index 49bfc9e..ff88c96 100644
--- a/core/src/main/java/org/apache/calcite/util/CalciteParserException.java
+++ b/core/src/main/java/org/apache/calcite/util/CalciteParserException.java
@@ -14,14 +14,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.util14;
+package org.apache.calcite.util;
 
 /**
- * This is a tagging interface to allow a {@link
- * org.eigenbase.sql.parser.SqlParseException} to be identified without adding a
+ * Tagging interface to allow a
+ * {@link org.apache.calcite.sql.parser.SqlParseException}
+ * to be identified without adding a
  * dependency on it from client-side code.
  */
-public interface EigenbaseParserException {
+public interface CalciteParserException {
 }
 
-// End EigenbaseParserException.java
+// End CalciteParserException.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/CalciteValidatorException.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/CalciteValidatorException.java b/core/src/main/java/org/apache/calcite/util/CalciteValidatorException.java
index b03fb2b..2adc473 100644
--- a/core/src/main/java/org/apache/calcite/util/CalciteValidatorException.java
+++ b/core/src/main/java/org/apache/calcite/util/CalciteValidatorException.java
@@ -14,14 +14,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.util14;
+package org.apache.calcite.util;
 
 /**
- * This is a tagging interface to allow {@link
- * org.eigenbase.sql.validate.SqlValidatorException} to be identified without
+ * Tagging interface to allow a
+ * {@link org.apache.calcite.sql.validate.SqlValidatorException}
+ * to be identified without
  * adding a dependency on it from client-side code.
  */
-public interface EigenbaseValidatorException {
+public interface CalciteValidatorException {
 }
 
-// End EigenbaseValidatorException.java
+// End CalciteValidatorException.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/CancelFlag.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/CancelFlag.java b/core/src/main/java/org/apache/calcite/util/CancelFlag.java
index b088efe..b1f57b7 100644
--- a/core/src/main/java/org/apache/calcite/util/CancelFlag.java
+++ b/core/src/main/java/org/apache/calcite/util/CancelFlag.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.util;
+package org.apache.calcite.util;
 
 /**
  * CancelFlag is used to post and check cancellation requests.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/util/CastingList.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/CastingList.java b/core/src/main/java/org/apache/calcite/util/CastingList.java
index e9583cb..8bcf85c 100644
--- a/core/src/main/java/org/apache/calcite/util/CastingList.java
+++ b/core/src/main/java/org/apache/calcite/util/CastingList.java
@@ -14,19 +14,22 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.util;
+package org.apache.calcite.util;
 
-import java.util.*;
+import java.util.AbstractList;
+import java.util.List;
 
 /**
  * Converts a list whose members are automatically down-cast to a given type.
  *
  * <p>If a member of the backing list is not an instanceof <code>E</code>, the
- * accessing method (such as {@link List#get}) will throw a {@link
- * ClassCastException}.
+ * accessing method (such as {@link List#get}) will throw a
+ * {@link ClassCastException}.
  *
  * <p>All modifications are automatically written to the backing list. Not
  * synchronized.
+ *
+ * @param <E> Element type
  */
 public class CastingList<E> extends AbstractList<E> implements List<E> {
   //~ Instance fields --------------------------------------------------------


[07/58] [abbrv] [partial] incubator-calcite git commit: [CALCITE-306] Standardize code style for "import package.*; "

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
----------------------------------------------------------------------
diff --git a/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml b/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
index da967a8..d017a36 100644
--- a/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
+++ b/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
@@ -22,21 +22,21 @@ limitations under the License.
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-AggregateRel(group=[{0, 1}])
-  UnionRel(all=[true])
-    ProjectRel(DEPTNO=[$0], NAME=[$1])
-      TableAccessRel(table=[[CATALOG, SALES, DEPT]])
-    ProjectRel(DEPTNO=[$0], NAME=[$1])
-      TableAccessRel(table=[[CATALOG, SALES, DEPT]])
+LogicalAggregate(group=[{0, 1}])
+  LogicalUnion(all=[true])
+    LogicalProject(DEPTNO=[$0], NAME=[$1])
+      LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+    LogicalProject(DEPTNO=[$0], NAME=[$1])
+      LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-UnionRel(all=[false])
-  ProjectRel(DEPTNO=[$0], NAME=[$1])
-    TableAccessRel(table=[[CATALOG, SALES, DEPT]])
-  ProjectRel(DEPTNO=[$0], NAME=[$1])
-    TableAccessRel(table=[[CATALOG, SALES, DEPT]])
+LogicalUnion(all=[false])
+  LogicalProject(DEPTNO=[$0], NAME=[$1])
+    LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+  LogicalProject(DEPTNO=[$0], NAME=[$1])
+    LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
         </Resource>
     </TestCase>
@@ -46,19 +46,19 @@ UnionRel(all=[false])
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-ProjectRel(EXPR$0=[1])
-  JoinRel(condition=[=($7, $9)], joinType=[inner])
-    TableAccessRel(table=[[CATALOG, SALES, EMP]])
-    TableAccessRel(table=[[CATALOG, SALES, DEPT]])
+LogicalProject(EXPR$0=[1])
+  LogicalJoin(condition=[=($7, $9)], joinType=[inner])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-ProjectRel(EXPR$0=[1])
-  FilterRel(condition=[=($7, $9)])
-    JoinRel(condition=[true], joinType=[inner])
-      TableAccessRel(table=[[CATALOG, SALES, EMP]])
-      TableAccessRel(table=[[CATALOG, SALES, DEPT]])
+LogicalProject(EXPR$0=[1])
+  LogicalFilter(condition=[=($7, $9)])
+    LogicalJoin(condition=[true], joinType=[inner])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+      LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
         </Resource>
     </TestCase>
@@ -68,20 +68,20 @@ ProjectRel(EXPR$0=[1])
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-ProjectRel(EXPR$0=[1])
-  JoinRel(condition=[=($7, $9)], joinType=[inner])
-    TableAccessRel(table=[[CATALOG, SALES, EMP]])
-    TableAccessRel(table=[[CATALOG, SALES, DEPT]])
+LogicalProject(EXPR$0=[1])
+  LogicalJoin(condition=[=($7, $9)], joinType=[inner])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-ProjectRel(EXPR$0=[1])
-  JoinRel(condition=[=($7, $9)], joinType=[inner], semiJoinDone=[true])
-    SemiJoinRel(condition=[=($7, $9)], joinType=[inner])
-      TableAccessRel(table=[[CATALOG, SALES, EMP]])
-      TableAccessRel(table=[[CATALOG, SALES, DEPT]])
-    TableAccessRel(table=[[CATALOG, SALES, DEPT]])
+LogicalProject(EXPR$0=[1])
+  LogicalJoin(condition=[=($7, $9)], joinType=[inner], semiJoinDone=[true])
+    SemiJoin(condition=[=($7, $9)], joinType=[inner])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+      LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+    LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
         </Resource>
     </TestCase>
@@ -114,21 +114,21 @@ ProjectRel(EXPR$0=[1])
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-ProjectRel(EXPR$0=[1])
-  FilterRel(condition=[=($1, 'Charlie')])
-    JoinRel(condition=[=($0, $9)], joinType=[full])
-      TableAccessRel(table=[[CATALOG, SALES, DEPT]])
-      TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(EXPR$0=[1])
+  LogicalFilter(condition=[=($1, 'Charlie')])
+    LogicalJoin(condition=[=($0, $9)], joinType=[full])
+      LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-ProjectRel(EXPR$0=[1])
-  ProjectRel(DEPTNO=[CAST($0):INTEGER], NAME=[CAST($1):VARCHAR(10) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary"], EMPNO=[$2], ENAME=[$3], JOB=[$4], MGR=[$5], HIREDATE=[$6], SAL=[$7], COMM=[$8], DEPTNO0=[$9], SLACKER=[$10])
-    JoinRel(condition=[=($0, $9)], joinType=[left])
-      FilterRel(condition=[=($1, 'Charlie')])
-        TableAccessRel(table=[[CATALOG, SALES, DEPT]])
-      TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(EXPR$0=[1])
+  LogicalProject(DEPTNO=[CAST($0):INTEGER], NAME=[CAST($1):VARCHAR(10) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary"], EMPNO=[$2], ENAME=[$3], JOB=[$4], MGR=[$5], HIREDATE=[$6], SAL=[$7], COMM=[$8], DEPTNO0=[$9], SLACKER=[$10])
+    LogicalJoin(condition=[=($0, $9)], joinType=[left])
+      LogicalFilter(condition=[=($1, 'Charlie')])
+        LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -138,21 +138,21 @@ ProjectRel(EXPR$0=[1])
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-ProjectRel(EXPR$0=[1])
-  FilterRel(condition=[>($7, 100)])
-    JoinRel(condition=[=($0, $9)], joinType=[full])
-      TableAccessRel(table=[[CATALOG, SALES, DEPT]])
-      TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(EXPR$0=[1])
+  LogicalFilter(condition=[>($7, 100)])
+    LogicalJoin(condition=[=($0, $9)], joinType=[full])
+      LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-ProjectRel(EXPR$0=[1])
-  ProjectRel(DEPTNO=[$0], NAME=[$1], EMPNO=[CAST($2):INTEGER], ENAME=[CAST($3):VARCHAR(20) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary"], JOB=[CAST($4):VARCHAR(10) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary"], MGR=[$5], HIREDATE=[CAST($6):TIMESTAMP(0)], SAL=[CAST($7):INTEGER], COMM=[CAST($8):INTEGER], DEPTNO0=[CAST($9):INTEGER], SLACKER=[CAST($10):BOOLEAN])
-    JoinRel(condition=[=($0, $9)], joinType=[right])
-      TableAccessRel(table=[[CATALOG, SALES, DEPT]])
-      FilterRel(condition=[>($5, 100)])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(EXPR$0=[1])
+  LogicalProject(DEPTNO=[$0], NAME=[$1], EMPNO=[CAST($2):INTEGER], ENAME=[CAST($3):VARCHAR(20) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary"], JOB=[CAST($4):VARCHAR(10) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary"], MGR=[$5], HIREDATE=[CAST($6):TIMESTAMP(0)], SAL=[CAST($7):INTEGER], COMM=[CAST($8):INTEGER], DEPTNO0=[CAST($9):INTEGER], SLACKER=[CAST($10):BOOLEAN])
+    LogicalJoin(condition=[=($0, $9)], joinType=[right])
+      LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+      LogicalFilter(condition=[>($5, 100)])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -162,22 +162,22 @@ ProjectRel(EXPR$0=[1])
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-ProjectRel(EXPR$0=[1])
-  FilterRel(condition=[AND(=($1, 'Charlie'), >($7, 100))])
-    JoinRel(condition=[=($0, $9)], joinType=[full])
-      TableAccessRel(table=[[CATALOG, SALES, DEPT]])
-      TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(EXPR$0=[1])
+  LogicalFilter(condition=[AND(=($1, 'Charlie'), >($7, 100))])
+    LogicalJoin(condition=[=($0, $9)], joinType=[full])
+      LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-ProjectRel(EXPR$0=[1])
-  ProjectRel(DEPTNO=[CAST($0):INTEGER], NAME=[CAST($1):VARCHAR(10) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary"], EMPNO=[CAST($2):INTEGER], ENAME=[CAST($3):VARCHAR(20) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary"], JOB=[CAST($4):VARCHAR(10) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary"], MGR=[$5], HIREDATE=[CAST($6):TIMESTAMP(0)], SAL=[CAST($7):INTEGER], COMM=[CAST($8):INTEGER], DEPTNO0=[CAST($9):INTEGER], SLACKER=[CAST($10):BOOLEAN])
-    JoinRel(condition=[=($0, $9)], joinType=[inner])
-      FilterRel(condition=[=($1, 'Charlie')])
-        TableAccessRel(table=[[CATALOG, SALES, DEPT]])
-      FilterRel(condition=[>($5, 100)])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(EXPR$0=[1])
+  LogicalProject(DEPTNO=[CAST($0):INTEGER], NAME=[CAST($1):VARCHAR(10) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary"], EMPNO=[CAST($2):INTEGER], ENAME=[CAST($3):VARCHAR(20) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary"], JOB=[CAST($4):VARCHAR(10) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary"], MGR=[$5], HIREDATE=[CAST($6):TIMESTAMP(0)], SAL=[CAST($7):INTEGER], COMM=[CAST($8):INTEGER], DEPTNO0=[CAST($9):INTEGER], SLACKER=[CAST($10):BOOLEAN])
+    LogicalJoin(condition=[=($0, $9)], joinType=[inner])
+      LogicalFilter(condition=[=($1, 'Charlie')])
+        LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+      LogicalFilter(condition=[>($5, 100)])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -187,21 +187,21 @@ ProjectRel(EXPR$0=[1])
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-ProjectRel(EXPR$0=[1])
-  FilterRel(condition=[>($7, 100)])
-    JoinRel(condition=[=($0, $9)], joinType=[left])
-      TableAccessRel(table=[[CATALOG, SALES, DEPT]])
-      TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(EXPR$0=[1])
+  LogicalFilter(condition=[>($7, 100)])
+    LogicalJoin(condition=[=($0, $9)], joinType=[left])
+      LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-ProjectRel(EXPR$0=[1])
-  ProjectRel(DEPTNO=[$0], NAME=[$1], EMPNO=[CAST($2):INTEGER], ENAME=[CAST($3):VARCHAR(20) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary"], JOB=[CAST($4):VARCHAR(10) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary"], MGR=[$5], HIREDATE=[CAST($6):TIMESTAMP(0)], SAL=[CAST($7):INTEGER], COMM=[CAST($8):INTEGER], DEPTNO0=[CAST($9):INTEGER], SLACKER=[CAST($10):BOOLEAN])
-    JoinRel(condition=[=($0, $9)], joinType=[inner])
-      TableAccessRel(table=[[CATALOG, SALES, DEPT]])
-      FilterRel(condition=[>($5, 100)])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(EXPR$0=[1])
+  LogicalProject(DEPTNO=[$0], NAME=[$1], EMPNO=[CAST($2):INTEGER], ENAME=[CAST($3):VARCHAR(20) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary"], JOB=[CAST($4):VARCHAR(10) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary"], MGR=[$5], HIREDATE=[CAST($6):TIMESTAMP(0)], SAL=[CAST($7):INTEGER], COMM=[CAST($8):INTEGER], DEPTNO0=[CAST($9):INTEGER], SLACKER=[CAST($10):BOOLEAN])
+    LogicalJoin(condition=[=($0, $9)], joinType=[inner])
+      LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+      LogicalFilter(condition=[>($5, 100)])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -211,21 +211,21 @@ ProjectRel(EXPR$0=[1])
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-ProjectRel(EXPR$0=[1])
-  FilterRel(condition=[=($1, 'Charlie')])
-    JoinRel(condition=[=($0, $9)], joinType=[right])
-      TableAccessRel(table=[[CATALOG, SALES, DEPT]])
-      TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(EXPR$0=[1])
+  LogicalFilter(condition=[=($1, 'Charlie')])
+    LogicalJoin(condition=[=($0, $9)], joinType=[right])
+      LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-ProjectRel(EXPR$0=[1])
-  ProjectRel(DEPTNO=[CAST($0):INTEGER], NAME=[CAST($1):VARCHAR(10) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary"], EMPNO=[$2], ENAME=[$3], JOB=[$4], MGR=[$5], HIREDATE=[$6], SAL=[$7], COMM=[$8], DEPTNO0=[$9], SLACKER=[$10])
-    JoinRel(condition=[=($0, $9)], joinType=[inner])
-      FilterRel(condition=[=($1, 'Charlie')])
-        TableAccessRel(table=[[CATALOG, SALES, DEPT]])
-      TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(EXPR$0=[1])
+  LogicalProject(DEPTNO=[CAST($0):INTEGER], NAME=[CAST($1):VARCHAR(10) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary"], EMPNO=[$2], ENAME=[$3], JOB=[$4], MGR=[$5], HIREDATE=[$6], SAL=[$7], COMM=[$8], DEPTNO0=[$9], SLACKER=[$10])
+    LogicalJoin(condition=[=($0, $9)], joinType=[inner])
+      LogicalFilter(condition=[=($1, 'Charlie')])
+        LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -237,26 +237,26 @@ ProjectRel(EXPR$0=[1])
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-ProjectRel(DEPTNO=[$0], NAME=[$1])
-  FilterRel(condition=[<=($0, 10)])
-    ProjectRel(DEPTNO=[$0], NAME=[$1])
-      SemiJoinRel(condition=[=($2, $3)], joinType=[inner])
-        ProjectRel($f0=[$0], $f1=[$1], $f2=[$0])
-          TableAccessRel(table=[[CATALOG, SALES, DEPT]])
-        ProjectRel(DEPTNO=[$7])
-          TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(DEPTNO=[$0], NAME=[$1])
+  LogicalFilter(condition=[<=($0, 10)])
+    LogicalProject(DEPTNO=[$0], NAME=[$1])
+      SemiJoin(condition=[=($2, $3)], joinType=[inner])
+        LogicalProject($f0=[$0], $f1=[$1], $f2=[$0])
+          LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+        LogicalProject(DEPTNO=[$7])
+          LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-ProjectRel(DEPTNO=[$0], NAME=[$1])
-  ProjectRel(DEPTNO=[$0], NAME=[$1])
-    SemiJoinRel(condition=[=($2, $3)], joinType=[inner])
-      FilterRel(condition=[<=($0, 10)])
-        ProjectRel($f0=[$0], $f1=[$1], $f2=[$0])
-          TableAccessRel(table=[[CATALOG, SALES, DEPT]])
-      ProjectRel(DEPTNO=[$7])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(DEPTNO=[$0], NAME=[$1])
+  LogicalProject(DEPTNO=[$0], NAME=[$1])
+    SemiJoin(condition=[=($2, $3)], joinType=[inner])
+      LogicalFilter(condition=[<=($0, 10)])
+        LogicalProject($f0=[$0], $f1=[$1], $f2=[$0])
+          LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+      LogicalProject(DEPTNO=[$7])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -266,20 +266,20 @@ ProjectRel(DEPTNO=[$0], NAME=[$1])
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-ProjectRel(DNAME=[$0], C=[$1])
-  FilterRel(condition=[=($0, 'Charlie')])
-    AggregateRel(group=[{0}], C=[COUNT()])
-      ProjectRel(DNAME=[$1])
-        TableAccessRel(table=[[CATALOG, SALES, DEPT]])
+LogicalProject(DNAME=[$0], C=[$1])
+  LogicalFilter(condition=[=($0, 'Charlie')])
+    LogicalAggregate(group=[{0}], C=[COUNT()])
+      LogicalProject(DNAME=[$1])
+        LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-ProjectRel(DNAME=[$0], C=[$1])
-  AggregateRel(group=[{0}], C=[COUNT()])
-    FilterRel(condition=[=($0, 'Charlie')])
-      ProjectRel(DNAME=[$1])
-        TableAccessRel(table=[[CATALOG, SALES, DEPT]])
+LogicalProject(DNAME=[$0], C=[$1])
+  LogicalAggregate(group=[{0}], C=[COUNT()])
+    LogicalFilter(condition=[=($0, 'Charlie')])
+      LogicalProject(DNAME=[$1])
+        LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
         </Resource>
     </TestCase>
@@ -289,18 +289,18 @@ ProjectRel(DNAME=[$0], C=[$1])
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-AggregateRel(group=[{0}], EXPR$1=[MAX($0)], EXPR$2=[AVG($1)], EXPR$3=[MIN($0)])
-  ProjectRel(NAME=[$1], DEPTNO=[$0])
-    TableAccessRel(table=[[CATALOG, SALES, DEPT]])
+LogicalAggregate(group=[{0}], EXPR$1=[MAX($0)], EXPR$2=[AVG($1)], EXPR$3=[MIN($0)])
+  LogicalProject(NAME=[$1], DEPTNO=[$0])
+    LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-ProjectRel(NAME=[$0], EXPR$1=[$1], EXPR$2=[CAST(/($2, $3)):INTEGER NOT NULL], EXPR$3=[$4])
-  ProjectRel(NAME=[$0], EXPR$1=[$1], $f2=[$2], $f3=[$3], EXPR$3=[$4])
-    AggregateRel(group=[{0}], EXPR$1=[MAX($0)], agg#1=[$SUM0($1)], agg#2=[COUNT()], EXPR$3=[MIN($0)])
-      ProjectRel(NAME=[$1], DEPTNO=[$0])
-        TableAccessRel(table=[[CATALOG, SALES, DEPT]])
+LogicalProject(NAME=[$0], EXPR$1=[$1], EXPR$2=[CAST(/($2, $3)):INTEGER NOT NULL], EXPR$3=[$4])
+  LogicalProject(NAME=[$0], EXPR$1=[$1], $f2=[$2], $f3=[$3], EXPR$3=[$4])
+    LogicalAggregate(group=[{0}], EXPR$1=[MAX($0)], agg#1=[$SUM0($1)], agg#2=[COUNT()], EXPR$3=[MIN($0)])
+      LogicalProject(NAME=[$1], DEPTNO=[$0])
+        LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
         </Resource>
     </TestCase>
@@ -310,17 +310,17 @@ ProjectRel(NAME=[$0], EXPR$1=[$1], EXPR$2=[CAST(/($2, $3)):INTEGER NOT NULL], EX
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-ProjectRel(EXPR$0=[+($0, $7)])
-  FilterRel(condition=[AND(=($5, *(10, $6)), =(UPPER($1), 'FOO'))])
-    TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(EXPR$0=[+($0, $7)])
+  LogicalFilter(condition=[AND(=($5, *(10, $6)), =(UPPER($1), 'FOO'))])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-ProjectRel(EXPR$0=[+($0, $4)])
-  FilterRel(condition=[AND(=($2, *(10, $3)), =(UPPER($1), 'FOO'))])
-    ProjectRel(EMPNO=[$0], ENAME=[$1], SAL=[$5], COMM=[$6], DEPTNO=[$7])
-      TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(EXPR$0=[+($0, $4)])
+  LogicalFilter(condition=[AND(=($2, *(10, $3)), =(UPPER($1), 'FOO'))])
+    LogicalProject(EMPNO=[$0], ENAME=[$1], SAL=[$5], COMM=[$6], DEPTNO=[$7])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -330,20 +330,20 @@ ProjectRel(EXPR$0=[+($0, $4)])
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-ProjectRel(EXPR$0=[+($5, $12)])
-  JoinRel(condition=[AND(=($1, $9), =($7, 10))], joinType=[inner])
-    TableAccessRel(table=[[CATALOG, SALES, EMP]])
-    TableAccessRel(table=[[CATALOG, SALES, BONUS]])
+LogicalProject(EXPR$0=[+($5, $12)])
+  LogicalJoin(condition=[AND(=($1, $9), =($7, 10))], joinType=[inner])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalTableScan(table=[[CATALOG, SALES, BONUS]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-ProjectRel(EXPR$0=[+($1, $4)])
-  JoinRel(condition=[AND(=($0, $3), =($2, 10))], joinType=[inner])
-    ProjectRel(ENAME=[$1], SAL=[$5], DEPTNO=[$7])
-      TableAccessRel(table=[[CATALOG, SALES, EMP]])
-    ProjectRel(ENAME=[$0], COMM=[$3])
-      TableAccessRel(table=[[CATALOG, SALES, BONUS]])
+LogicalProject(EXPR$0=[+($1, $4)])
+  LogicalJoin(condition=[AND(=($0, $3), =($2, 10))], joinType=[inner])
+    LogicalProject(ENAME=[$1], SAL=[$5], DEPTNO=[$7])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalProject(ENAME=[$0], COMM=[$3])
+      LogicalTableScan(table=[[CATALOG, SALES, BONUS]])
 ]]>
         </Resource>
     </TestCase>
@@ -353,23 +353,23 @@ ProjectRel(EXPR$0=[+($1, $4)])
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-ProjectRel(SAL=[$5])
-  UnionRel(all=[true])
-    ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-      TableAccessRel(table=[[CATALOG, SALES, EMP]])
-    ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-      TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(SAL=[$5])
+  LogicalUnion(all=[true])
+    LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-UnionRel(all=[true])
-  ProjectRel(SAL=[$5])
-    ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-      TableAccessRel(table=[[CATALOG, SALES, EMP]])
-  ProjectRel(SAL=[$5])
-    ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-      TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalUnion(all=[true])
+  LogicalProject(SAL=[$5])
+    LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+  LogicalProject(SAL=[$5])
+    LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -379,28 +379,28 @@ UnionRel(all=[true])
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-ProjectRel(SAL=[$5])
-  JoinRel(condition=[true], joinType=[inner])
-    UnionRel(all=[true])
-      ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
-      ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
-    TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(SAL=[$5])
+  LogicalJoin(condition=[true], joinType=[inner])
+    LogicalUnion(all=[true])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-ProjectRel(SAL=[$5])
-  UnionRel(all=[true])
-    JoinRel(condition=[true], joinType=[inner])
-      ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
-      TableAccessRel(table=[[CATALOG, SALES, EMP]])
-    JoinRel(condition=[true], joinType=[inner])
-      ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
-      TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(SAL=[$5])
+  LogicalUnion(all=[true])
+    LogicalJoin(condition=[true], joinType=[inner])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalJoin(condition=[true], joinType=[inner])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -410,28 +410,28 @@ ProjectRel(SAL=[$5])
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-ProjectRel(SAL=[$5])
-  JoinRel(condition=[true], joinType=[inner])
-    TableAccessRel(table=[[CATALOG, SALES, EMP]])
-    UnionRel(all=[true])
-      ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
-      ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(SAL=[$5])
+  LogicalJoin(condition=[true], joinType=[inner])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalUnion(all=[true])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-ProjectRel(SAL=[$5])
-  UnionRel(all=[true])
-    JoinRel(condition=[true], joinType=[inner])
-      TableAccessRel(table=[[CATALOG, SALES, EMP]])
-      ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
-    JoinRel(condition=[true], joinType=[inner])
-      TableAccessRel(table=[[CATALOG, SALES, EMP]])
-      ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(SAL=[$5])
+  LogicalUnion(all=[true])
+    LogicalJoin(condition=[true], joinType=[inner])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalJoin(condition=[true], joinType=[inner])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -441,24 +441,24 @@ ProjectRel(SAL=[$5])
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-ProjectRel(EXPR$0=[+(1, 2)], EXPR$1=[+($0, +(3, 4))], EXPR$2=[+(+(5, 6), $0)], EXPR$3=[null], EXPR$4=[CASE(IS NOT NULL(2), 2, null)], EXPR$5=[ROW(+(7, 8))])
-  FilterRel(condition=[AND(=($0, +(7, 8)), =($0, CASE(IS NOT NULL(2), 2, null)))])
-    ProjectRel(DEPTNO=[$0], NAME=[$1], EMPNO=[$2], ENAME=[$3], JOB=[$4], MGR=[$5], HIREDATE=[$6], SAL=[$7], COMM=[$8], DEPTNO0=[$9], SLACKER=[$10])
-      JoinRel(condition=[=($0, $11)], joinType=[inner])
-        TableAccessRel(table=[[CATALOG, SALES, DEPT]])
-        ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], $f9=[+($7, -(5, 5))])
-          TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(EXPR$0=[+(1, 2)], EXPR$1=[+($0, +(3, 4))], EXPR$2=[+(+(5, 6), $0)], EXPR$3=[null], EXPR$4=[CASE(IS NOT NULL(2), 2, null)], EXPR$5=[ROW(+(7, 8))])
+  LogicalFilter(condition=[AND(=($0, +(7, 8)), =($0, CASE(IS NOT NULL(2), 2, null)))])
+    LogicalProject(DEPTNO=[$0], NAME=[$1], EMPNO=[$2], ENAME=[$3], JOB=[$4], MGR=[$5], HIREDATE=[$6], SAL=[$7], COMM=[$8], DEPTNO0=[$9], SLACKER=[$10])
+      LogicalJoin(condition=[=($0, $11)], joinType=[inner])
+        LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+        LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], $f9=[+($7, -(5, 5))])
+          LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-ProjectRel(EXPR$0=[3], EXPR$1=[+($0, 7)], EXPR$2=[+(11, $0)], EXPR$3=[null], EXPR$4=[CAST(2):INTEGER], EXPR$5=[ROW(15)])
-  FilterRel(condition=[AND(=($0, 15), =($0, CAST(2):INTEGER))])
-    ProjectRel(DEPTNO=[$0], NAME=[$1], EMPNO=[$2], ENAME=[$3], JOB=[$4], MGR=[$5], HIREDATE=[$6], SAL=[$7], COMM=[$8], DEPTNO0=[$9], SLACKER=[$10])
-      JoinRel(condition=[=($0, $11)], joinType=[inner])
-        TableAccessRel(table=[[CATALOG, SALES, DEPT]])
-        ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], $f9=[+($7, 0)])
-          TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(EXPR$0=[3], EXPR$1=[+($0, 7)], EXPR$2=[+(11, $0)], EXPR$3=[null], EXPR$4=[CAST(2):INTEGER], EXPR$5=[ROW(15)])
+  LogicalFilter(condition=[AND(=($0, 15), =($0, CAST(2):INTEGER))])
+    LogicalProject(DEPTNO=[$0], NAME=[$1], EMPNO=[$2], ENAME=[$3], JOB=[$4], MGR=[$5], HIREDATE=[$6], SAL=[$7], COMM=[$8], DEPTNO0=[$9], SLACKER=[$10])
+      LogicalJoin(condition=[=($0, $11)], joinType=[inner])
+        LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+        LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], $f9=[+($7, 0)])
+          LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -468,15 +468,15 @@ ProjectRel(EXPR$0=[3], EXPR$1=[+($0, 7)], EXPR$2=[+(11, $0)], EXPR$3=[null], EXP
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-ProjectRel(EXPR$0=[$0], EXPR$1=[$1])
-  FilterRel(condition=[>(+(1, 2), +(3, null))])
-    ValuesRel(tuples=[[{ 1, 2 }]])
+LogicalProject(EXPR$0=[$0], EXPR$1=[$1])
+  LogicalFilter(condition=[>(+(1, 2), +(3, null))])
+    LogicalValues(tuples=[[{ 1, 2 }]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-ProjectRel(EXPR$0=[$0], EXPR$1=[$1])
-  EmptyRel
+LogicalProject(EXPR$0=[$0], EXPR$1=[$1])
+  Empty
 ]]>
         </Resource>
     </TestCase>
@@ -486,15 +486,15 @@ ProjectRel(EXPR$0=[$0], EXPR$1=[$1])
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-ProjectRel(EXPR$0=[$0], EXPR$1=[$1])
-  FilterRel(condition=[false])
-    ValuesRel(tuples=[[{ 1, 2 }]])
+LogicalProject(EXPR$0=[$0], EXPR$1=[$1])
+  LogicalFilter(condition=[false])
+    LogicalValues(tuples=[[{ 1, 2 }]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-ProjectRel(EXPR$0=[$0], EXPR$1=[$1])
-  EmptyRel
+LogicalProject(EXPR$0=[$0], EXPR$1=[$1])
+  Empty
 ]]>
         </Resource>
     </TestCase>
@@ -504,15 +504,15 @@ ProjectRel(EXPR$0=[$0], EXPR$1=[$1])
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-ProjectRel(A=[$0], B=[$1])
-  FilterRel(condition=[<($0, 15)])
-    ValuesRel(tuples=[[{ 10, 'x' }, { 20, 'y' }]])
+LogicalProject(A=[$0], B=[$1])
+  LogicalFilter(condition=[<($0, 15)])
+    LogicalValues(tuples=[[{ 10, 'x' }, { 20, 'y' }]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-ProjectRel(A=[$0], B=[$1])
-  ValuesRel(tuples=[[{ 10, 'x' }]])
+LogicalProject(A=[$0], B=[$1])
+  LogicalValues(tuples=[[{ 10, 'x' }]])
 ]]>
         </Resource>
     </TestCase>
@@ -522,13 +522,13 @@ ProjectRel(A=[$0], B=[$1])
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-ProjectRel(EXPR$0=[+($0, $1)])
-  ValuesRel(tuples=[[{ 10, 1 }, { 20, 3 }]])
+LogicalProject(EXPR$0=[+($0, $1)])
+  LogicalValues(tuples=[[{ 10, 1 }, { 20, 3 }]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-ValuesRel(tuples=[[{ 11 }, { 23 }]])
+LogicalValues(tuples=[[{ 11 }, { 23 }]])
 ]]>
         </Resource>
     </TestCase>
@@ -538,14 +538,14 @@ ValuesRel(tuples=[[{ 11 }, { 23 }]])
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-ProjectRel(X=[+($0, $1)], B=[$1], A=[$0])
-  FilterRel(condition=[<(-($0, $1), 21)])
-    ValuesRel(tuples=[[{ 10, 1 }, { 30, 7 }, { 20, 3 }]])
+LogicalProject(X=[+($0, $1)], B=[$1], A=[$0])
+  LogicalFilter(condition=[<(-($0, $1), 21)])
+    LogicalValues(tuples=[[{ 10, 1 }, { 30, 7 }, { 20, 3 }]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-ValuesRel(tuples=[[{ 11, 1, 10 }, { 23, 3, 20 }]])
+LogicalValues(tuples=[[{ 11, 1, 10 }, { 23, 3, 20 }]])
 ]]>
         </Resource>
     </TestCase>
@@ -555,14 +555,14 @@ ValuesRel(tuples=[[{ 11, 1, 10 }, { 23, 3, 20 }]])
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-ProjectRel(X=[+($0, $1)], B=[$1], A=[$0])
-  FilterRel(condition=[<(-($0, $1), 0)])
-    ValuesRel(tuples=[[{ 10, 1 }, { 30, 7 }]])
+LogicalProject(X=[+($0, $1)], B=[$1], A=[$0])
+  LogicalFilter(condition=[<(-($0, $1), 0)])
+    LogicalValues(tuples=[[{ 10, 1 }, { 30, 7 }]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-EmptyRel
+Empty
 ]]>
         </Resource>
     </TestCase>
@@ -577,19 +577,19 @@ where x + y > 30]]>
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-ProjectRel(X=[$0], Y=[$1])
-  FilterRel(condition=[>(+($0, $1), 30)])
-    UnionRel(all=[true])
-      ProjectRel(X=[$0], Y=[$1])
-        ValuesRel(tuples=[[{ 10, 1 }, { 30, 3 }]])
-      ProjectRel(EXPR$0=[$0], EXPR$1=[$1])
-        ValuesRel(tuples=[[{ 20, 2 }]])
+LogicalProject(X=[$0], Y=[$1])
+  LogicalFilter(condition=[>(+($0, $1), 30)])
+    LogicalUnion(all=[true])
+      LogicalProject(X=[$0], Y=[$1])
+        LogicalValues(tuples=[[{ 10, 1 }, { 30, 3 }]])
+      LogicalProject(EXPR$0=[$0], EXPR$1=[$1])
+        LogicalValues(tuples=[[{ 20, 2 }]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-ProjectRel(X=[$0], Y=[$1])
-  ValuesRel(tuples=[[{ 30, 3 }]])
+LogicalProject(X=[$0], Y=[$1])
+  LogicalValues(tuples=[[{ 30, 3 }]])
 ]]>
         </Resource>
     </TestCase>
@@ -599,26 +599,26 @@ ProjectRel(X=[$0], Y=[$1])
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-ProjectRel(EXPR$0=[CAST($1):VARCHAR(128) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary" NOT NULL], EXPR$1=[CAST($2):INTEGER NOT NULL])
-  FilterRel(condition=[=(CAST(CAST($4):VARCHAR(1) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary" NOT NULL):VARCHAR(7) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary" NOT NULL, 'Manager')])
-    ProjectRel(DEPTNO=[$0], NAME=[$1], EMPNO=[$3], ENAME=[$4], JOB=[$5], MGR=[$6], HIREDATE=[$7], SAL=[$8], COMM=[$9], DEPTNO0=[$10], SLACKER=[$11])
-      JoinRel(condition=[=($2, $12)], joinType=[inner])
-        ProjectRel(DEPTNO=[$0], NAME=[$1], $f2=[CAST($0):INTEGER NOT NULL])
-          TableAccessRel(table=[[CATALOG, SALES, DEPT]])
-        ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], $f9=[CAST($7):INTEGER NOT NULL])
-          TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(EXPR$0=[CAST($1):VARCHAR(128) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary" NOT NULL], EXPR$1=[CAST($2):INTEGER NOT NULL])
+  LogicalFilter(condition=[=(CAST(CAST($4):VARCHAR(1) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary" NOT NULL):VARCHAR(7) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary" NOT NULL, 'Manager')])
+    LogicalProject(DEPTNO=[$0], NAME=[$1], EMPNO=[$3], ENAME=[$4], JOB=[$5], MGR=[$6], HIREDATE=[$7], SAL=[$8], COMM=[$9], DEPTNO0=[$10], SLACKER=[$11])
+      LogicalJoin(condition=[=($2, $12)], joinType=[inner])
+        LogicalProject(DEPTNO=[$0], NAME=[$1], $f2=[CAST($0):INTEGER NOT NULL])
+          LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+        LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], $f9=[CAST($7):INTEGER NOT NULL])
+          LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-ProjectRel(EXPR$0=[CAST($1):VARCHAR(128) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary" NOT NULL], EXPR$1=[$2])
-  FilterRel(condition=[=(CAST(CAST($4):VARCHAR(1) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary" NOT NULL):VARCHAR(7) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary" NOT NULL, 'Manager')])
-    ProjectRel(DEPTNO=[$0], NAME=[$1], EMPNO=[$3], ENAME=[$4], JOB=[$5], MGR=[$6], HIREDATE=[$7], SAL=[$8], COMM=[$9], DEPTNO0=[$10], SLACKER=[$11])
-      JoinRel(condition=[=($2, $12)], joinType=[inner])
-        ProjectRel(DEPTNO=[$0], NAME=[$1], $f2=[$0])
-          TableAccessRel(table=[[CATALOG, SALES, DEPT]])
-        ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], $f9=[$7])
-          TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(EXPR$0=[CAST($1):VARCHAR(128) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary" NOT NULL], EXPR$1=[$2])
+  LogicalFilter(condition=[=(CAST(CAST($4):VARCHAR(1) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary" NOT NULL):VARCHAR(7) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary" NOT NULL, 'Manager')])
+    LogicalProject(DEPTNO=[$0], NAME=[$1], EMPNO=[$3], ENAME=[$4], JOB=[$5], MGR=[$6], HIREDATE=[$7], SAL=[$8], COMM=[$9], DEPTNO0=[$10], SLACKER=[$11])
+      LogicalJoin(condition=[=($2, $12)], joinType=[inner])
+        LogicalProject(DEPTNO=[$0], NAME=[$1], $f2=[$0])
+          LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+        LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], $f9=[$7])
+          LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -661,16 +661,16 @@ ProjectRel(ENAME=[$0], EXPR$1=[$1], EXPR$2=[CAST($2):BIGINT NOT NULL])
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-  FilterRel(condition=[=(CAST(+($0, /(10, 2))):INTEGER NOT NULL, 13)])
-    TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+  LogicalFilter(condition=[=(CAST(+($0, /(10, 2))):INTEGER NOT NULL, 13)])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-  FilterRel(condition=[=(+($0, 5), 13)])
-    TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+  LogicalFilter(condition=[=(+($0, 5), 13)])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -690,23 +690,23 @@ ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5],
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-ProjectRel(U=[$0], S=[$1])
-  FilterRel(condition=[=($0, 'TABLE')])
-    ProjectRel(U=[UPPER(||(SUBSTRING($0, 1, 2), SUBSTRING($0, 3)))], S=[SUBSTRING($0, 1, 1)])
-      UnionRel(all=[false])
-        UnionRel(all=[false])
-          ProjectRel(X=['table'])
-            ValuesRel(tuples=[[{ true }]])
-          ProjectRel(EXPR$0=['view'])
-            ValuesRel(tuples=[[{ true }]])
-        ProjectRel(EXPR$0=['foreign table'])
-          ValuesRel(tuples=[[{ true }]])
+LogicalProject(U=[$0], S=[$1])
+  LogicalFilter(condition=[=($0, 'TABLE')])
+    LogicalProject(U=[UPPER(||(SUBSTRING($0, 1, 2), SUBSTRING($0, 3)))], S=[SUBSTRING($0, 1, 1)])
+      LogicalUnion(all=[false])
+        LogicalUnion(all=[false])
+          LogicalProject(X=['table'])
+            LogicalValues(tuples=[[{ true }]])
+          LogicalProject(EXPR$0=['view'])
+            LogicalValues(tuples=[[{ true }]])
+        LogicalProject(EXPR$0=['foreign table'])
+          LogicalValues(tuples=[[{ true }]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-CalcRel(expr#0=[{inputs}], expr#1=['TABLE'], expr#2=['t'], U=[$t1], S=[$t2])
-  ValuesRel(tuples=[[{ true }]])
+LogicalCalc(expr#0=[{inputs}], expr#1=['TABLE'], expr#2=['t'], U=[$t1], S=[$t2])
+  LogicalValues(tuples=[[{ true }]])
 ]]>
         </Resource>
     </TestCase>
@@ -716,16 +716,16 @@ CalcRel(expr#0=[{inputs}], expr#1=['TABLE'], expr#2=['t'], U=[$t1], S=[$t2])
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-ProjectRel(EXPR$0=[CAST(CASE(IS NULL($1), IS NULL($0), IS NULL($0), IS NULL($1), =($1, $0))):BOOLEAN NOT NULL])
-  ProjectRel(EXPR$0=[2], EXPR$1=[null])
-    OneRowRel
+LogicalProject(EXPR$0=[CAST(CASE(IS NULL($1), IS NULL($0), IS NULL($0), IS NULL($1), =($1, $0))):BOOLEAN NOT NULL])
+  LogicalProject(EXPR$0=[2], EXPR$1=[null])
+    LogicalOneRow
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-ProjectRel(EXPR$0=[CAST(CASE(IS NULL($1), IS NULL($0), IS NULL($0), IS NULL($1), =($1, $0))):BOOLEAN NOT NULL])
-  ProjectRel(EXPR$0=[2], EXPR$1=[null])
-    OneRowRel
+LogicalProject(EXPR$0=[CAST(CASE(IS NULL($1), IS NULL($0), IS NULL($0), IS NULL($1), =($1, $0))):BOOLEAN NOT NULL])
+  LogicalProject(EXPR$0=[2], EXPR$1=[null])
+    LogicalOneRow
 ]]>
         </Resource>
     </TestCase>
@@ -737,18 +737,18 @@ join dept using (deptno)]]>
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$9], NAME=[$10])
-  JoinRel(condition=[=($7, $9)], joinType=[inner])
-    ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-      FilterRel(condition=[false])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
-    TableAccessRel(table=[[CATALOG, SALES, DEPT]])
+LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$9], NAME=[$10])
+  LogicalJoin(condition=[=($7, $9)], joinType=[inner])
+    LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+      LogicalFilter(condition=[false])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$9], NAME=[$10])
-  EmptyRel
+LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$9], NAME=[$10])
+  Empty
 ]]>
         </Resource>
     </TestCase>
@@ -760,20 +760,20 @@ right join dept using (deptno)]]>
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$9], NAME=[$10])
-  JoinRel(condition=[=($7, $9)], joinType=[right])
-    ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-      FilterRel(condition=[false])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
-    TableAccessRel(table=[[CATALOG, SALES, DEPT]])
+LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$9], NAME=[$10])
+  LogicalJoin(condition=[=($7, $9)], joinType=[right])
+    LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+      LogicalFilter(condition=[false])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$9], NAME=[$10])
-  JoinRel(condition=[=($7, $9)], joinType=[right])
-    EmptyRel
-    TableAccessRel(table=[[CATALOG, SALES, DEPT]])
+LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$9], NAME=[$10])
+  LogicalJoin(condition=[=($7, $9)], joinType=[right])
+    Empty
+    LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
         </Resource>
     </TestCase>
@@ -785,18 +785,18 @@ left join dept using (deptno)]]>
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$9], NAME=[$10])
-  JoinRel(condition=[=($7, $9)], joinType=[left])
-    ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-      FilterRel(condition=[false])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
-    TableAccessRel(table=[[CATALOG, SALES, DEPT]])
+LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$9], NAME=[$10])
+  LogicalJoin(condition=[=($7, $9)], joinType=[left])
+    LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+      LogicalFilter(condition=[false])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$9], NAME=[$10])
-  EmptyRel
+LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$9], NAME=[$10])
+  Empty
 ]]>
         </Resource>
     </TestCase>
@@ -823,17 +823,17 @@ EmptyRel
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-SortRel(sort0=[$7], dir0=[ASC])
-  ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-    FilterRel(condition=[false])
-      TableAccessRel(table=[[CATALOG, SALES, EMP]])
+Sort(sort0=[$7], dir0=[ASC])
+  LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+    LogicalFilter(condition=[false])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-SortRel(sort0=[$7], dir0=[ASC])
-  ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-    EmptyRel
+Sort(sort0=[$7], dir0=[ASC])
+  LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+    Empty
 ]]>
         </Resource>
     </TestCase>
@@ -843,14 +843,14 @@ SortRel(sort0=[$7], dir0=[ASC])
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-SortRel(sort0=[$7], dir0=[ASC], fetch=[0])
-  ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-    TableAccessRel(table=[[CATALOG, SALES, EMP]])
+Sort(sort0=[$7], dir0=[ASC], fetch=[0])
+  LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-EmptyRel
+Empty
 ]]>
         </Resource>
     </TestCase>
@@ -863,25 +863,25 @@ group by ename]]>
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-AggregateRel(group=[{0}], EXPR$1=[SUM($1)])
-  ProjectRel(ENAME=[$1], U=[$9])
-    UnionRel(all=[true])
-      ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], U=[2])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
-      ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], U=[3])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalAggregate(group=[{0}], EXPR$1=[SUM($1)])
+  LogicalProject(ENAME=[$1], U=[$9])
+    LogicalUnion(all=[true])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], U=[2])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], U=[3])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-AggregateRel(group=[{0}], EXPR$1=[SUM($1)])
-  UnionRel(all=[true])
-    AggregateRel(group=[{0}], EXPR$1=[SUM($1)])
-      ProjectRel(ENAME=[$1], U=[2])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
-    AggregateRel(group=[{0}], EXPR$1=[SUM($1)])
-      ProjectRel(ENAME=[$1], U=[3])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalAggregate(group=[{0}], EXPR$1=[SUM($1)])
+  LogicalUnion(all=[true])
+    LogicalAggregate(group=[{0}], EXPR$1=[SUM($1)])
+      LogicalProject(ENAME=[$1], U=[2])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalAggregate(group=[{0}], EXPR$1=[SUM($1)])
+      LogicalProject(ENAME=[$1], U=[3])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -894,25 +894,25 @@ group by ename]]>
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-AggregateRel(group=[{0}], EXPR$1=[SUM($1)])
-  ProjectRel(ENAME=[$1], U=[$9])
-    UnionRel(all=[true])
-      ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], U=[null])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
-      ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], U=[null])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalAggregate(group=[{0}], EXPR$1=[SUM($1)])
+  LogicalProject(ENAME=[$1], U=[$9])
+    LogicalUnion(all=[true])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], U=[null])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], U=[null])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-AggregateRel(group=[{0}], EXPR$1=[SUM($1)])
-  UnionRel(all=[true])
-    AggregateRel(group=[{0}], EXPR$1=[SUM($1)])
-      ProjectRel(ENAME=[$1], U=[null])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
-    AggregateRel(group=[{0}], EXPR$1=[SUM($1)])
-      ProjectRel(ENAME=[$1], U=[null])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalAggregate(group=[{0}], EXPR$1=[SUM($1)])
+  LogicalUnion(all=[true])
+    LogicalAggregate(group=[{0}], EXPR$1=[SUM($1)])
+      LogicalProject(ENAME=[$1], U=[null])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalAggregate(group=[{0}], EXPR$1=[SUM($1)])
+      LogicalProject(ENAME=[$1], U=[null])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -926,25 +926,25 @@ group by ename
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-AggregateRel(group=[{0}], EXPR$1=[SUM($1)])
-  ProjectRel(ENAME=[$1], MGR=[$3])
-    UnionRel(all=[true])
-      ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
-      ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalAggregate(group=[{0}], EXPR$1=[SUM($1)])
+  LogicalProject(ENAME=[$1], MGR=[$3])
+    LogicalUnion(all=[true])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-AggregateRel(group=[{0}], EXPR$1=[SUM($1)])
-  UnionRel(all=[true])
-    AggregateRel(group=[{0}], EXPR$1=[SUM($1)])
-      ProjectRel(ENAME=[$1], MGR=[$3])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
-    AggregateRel(group=[{0}], EXPR$1=[SUM($1)])
-      ProjectRel(ENAME=[$1], MGR=[$3])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalAggregate(group=[{0}], EXPR$1=[SUM($1)])
+  LogicalUnion(all=[true])
+    LogicalAggregate(group=[{0}], EXPR$1=[SUM($1)])
+      LogicalProject(ENAME=[$1], MGR=[$3])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalAggregate(group=[{0}], EXPR$1=[SUM($1)])
+      LogicalProject(ENAME=[$1], MGR=[$3])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -957,25 +957,25 @@ AggregateRel(group=[{0}], EXPR$1=[SUM($1)])
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-AggregateRel(group=[{}], EXPR$0=[SUM($0)])
-  ProjectRel(MGR=[$3])
-    UnionRel(all=[true])
-      ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
-      ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalAggregate(group=[{}], EXPR$0=[SUM($0)])
+  LogicalProject(MGR=[$3])
+    LogicalUnion(all=[true])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-AggregateRel(group=[{}], EXPR$0=[SUM($0)])
-  UnionRel(all=[true])
-    AggregateRel(group=[{}], EXPR$0=[SUM($0)])
-      ProjectRel(MGR=[$3])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
-    AggregateRel(group=[{}], EXPR$0=[SUM($0)])
-      ProjectRel(MGR=[$3])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalAggregate(group=[{}], EXPR$0=[SUM($0)])
+  LogicalUnion(all=[true])
+    LogicalAggregate(group=[{}], EXPR$0=[SUM($0)])
+      LogicalProject(MGR=[$3])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalAggregate(group=[{}], EXPR$0=[SUM($0)])
+      LogicalProject(MGR=[$3])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -988,25 +988,25 @@ group by ename]]>
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-AggregateRel(group=[{0}], EXPR$1=[COUNT()])
-  ProjectRel(ENAME=[$1])
-    UnionRel(all=[true])
-      ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
-      ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalAggregate(group=[{0}], EXPR$1=[COUNT()])
+  LogicalProject(ENAME=[$1])
+    LogicalUnion(all=[true])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-AggregateRel(group=[{0}], EXPR$1=[$SUM0($1)])
-  UnionRel(all=[true])
-    AggregateRel(group=[{0}], EXPR$1=[COUNT()])
-      ProjectRel(ENAME=[$1])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
-    AggregateRel(group=[{0}], EXPR$1=[COUNT()])
-      ProjectRel(ENAME=[$1])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalAggregate(group=[{0}], EXPR$1=[$SUM0($1)])
+  LogicalUnion(all=[true])
+    LogicalAggregate(group=[{0}], EXPR$1=[COUNT()])
+      LogicalProject(ENAME=[$1])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalAggregate(group=[{0}], EXPR$1=[COUNT()])
+      LogicalProject(ENAME=[$1])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -1019,25 +1019,25 @@ group by ename]]>
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-AggregateRel(group=[{0}], EXPR$1=[COUNT($1)])
-  ProjectRel(ENAME=[$1], MGR=[$3])
-    UnionRel(all=[true])
-      ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
-      ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalAggregate(group=[{0}], EXPR$1=[COUNT($1)])
+  LogicalProject(ENAME=[$1], MGR=[$3])
+    LogicalUnion(all=[true])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-AggregateRel(group=[{0}], EXPR$1=[$SUM0($1)])
-  UnionRel(all=[true])
-    AggregateRel(group=[{0}], EXPR$1=[COUNT($1)])
-      ProjectRel(ENAME=[$1], MGR=[$3])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
-    AggregateRel(group=[{0}], EXPR$1=[COUNT($1)])
-      ProjectRel(ENAME=[$1], MGR=[$3])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalAggregate(group=[{0}], EXPR$1=[$SUM0($1)])
+  LogicalUnion(all=[true])
+    LogicalAggregate(group=[{0}], EXPR$1=[COUNT($1)])
+      LogicalProject(ENAME=[$1], MGR=[$3])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalAggregate(group=[{0}], EXPR$1=[COUNT($1)])
+      LogicalProject(ENAME=[$1], MGR=[$3])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -1050,25 +1050,25 @@ group by ename]]>
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-AggregateRel(group=[{0}], EXPR$1=[MAX($1)])
-  ProjectRel(ENAME=[$1], MGR=[$3])
-    UnionRel(all=[true])
-      ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
-      ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalAggregate(group=[{0}], EXPR$1=[MAX($1)])
+  LogicalProject(ENAME=[$1], MGR=[$3])
+    LogicalUnion(all=[true])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-AggregateRel(group=[{0}], EXPR$1=[MAX($1)])
-  UnionRel(all=[true])
-    AggregateRel(group=[{0}], EXPR$1=[MAX($1)])
-      ProjectRel(ENAME=[$1], MGR=[$3])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
-    AggregateRel(group=[{0}], EXPR$1=[MAX($1)])
-      ProjectRel(ENAME=[$1], MGR=[$3])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalAggregate(group=[{0}], EXPR$1=[MAX($1)])
+  LogicalUnion(all=[true])
+    LogicalAggregate(group=[{0}], EXPR$1=[MAX($1)])
+      LogicalProject(ENAME=[$1], MGR=[$3])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalAggregate(group=[{0}], EXPR$1=[MAX($1)])
+      LogicalProject(ENAME=[$1], MGR=[$3])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -1081,25 +1081,25 @@ AggregateRel(group=[{0}], EXPR$1=[MAX($1)])
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-AggregateRel(group=[{0}], EXPR$1=[MIN($1)])
-  ProjectRel(ENAME=[$1], EMPNO=[$0])
-    UnionRel(all=[true])
-      ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
-      ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalAggregate(group=[{0}], EXPR$1=[MIN($1)])
+  LogicalProject(ENAME=[$1], EMPNO=[$0])
+    LogicalUnion(all=[true])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-AggregateRel(group=[{0}], EXPR$1=[MIN($1)])
-  UnionRel(all=[true])
-    AggregateRel(group=[{0}], EXPR$1=[MIN($1)])
-      ProjectRel(ENAME=[$1], EMPNO=[$0])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
-    AggregateRel(group=[{0}], EXPR$1=[MIN($1)])
-      ProjectRel(ENAME=[$1], EMPNO=[$0])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalAggregate(group=[{0}], EXPR$1=[MIN($1)])
+  LogicalUnion(all=[true])
+    LogicalAggregate(group=[{0}], EXPR$1=[MIN($1)])
+      LogicalProject(ENAME=[$1], EMPNO=[$0])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalAggregate(group=[{0}], EXPR$1=[MIN($1)])
+      LogicalProject(ENAME=[$1], EMPNO=[$0])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -1112,23 +1112,23 @@ group by ename]]>
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-AggregateRel(group=[{0}], EXPR$1=[AVG($1)])
-  ProjectRel(ENAME=[$1], EMPNO=[$0])
-    UnionRel(all=[true])
-      ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
-      ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalAggregate(group=[{0}], EXPR$1=[AVG($1)])
+  LogicalProject(ENAME=[$1], EMPNO=[$0])
+    LogicalUnion(all=[true])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-AggregateRel(group=[{0}], EXPR$1=[AVG($1)])
-  UnionRel(all=[true])
-    ProjectRel(ENAME=[$1], EMPNO=[$0])
-      TableAccessRel(table=[[CATALOG, SALES, EMP]])
-    ProjectRel(ENAME=[$1], EMPNO=[$0])
-      TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalAggregate(group=[{0}], EXPR$1=[AVG($1)])
+  LogicalUnion(all=[true])
+    LogicalProject(ENAME=[$1], EMPNO=[$0])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalProject(ENAME=[$1], EMPNO=[$0])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -1141,25 +1141,25 @@ group by ename]]>
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-AggregateRel(group=[{0}], EXPR$1=[SUM($1)], EXPR$2=[COUNT()], EXPR$3=[MIN($2)], EXPR$4=[MAX($1)])
-  ProjectRel(ENAME=[$1], EMPNO=[$0], DEPTNO=[$7])
-    UnionRel(all=[true])
-      ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
-      ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalAggregate(group=[{0}], EXPR$1=[SUM($1)], EXPR$2=[COUNT()], EXPR$3=[MIN($2)], EXPR$4=[MAX($1)])
+  LogicalProject(ENAME=[$1], EMPNO=[$0], DEPTNO=[$7])
+    LogicalUnion(all=[true])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-AggregateRel(group=[{0}], EXPR$1=[SUM($1)], EXPR$2=[$SUM0($2)], EXPR$3=[MIN($3)], EXPR$4=[MAX($4)])
-  UnionRel(all=[true])
-    AggregateRel(group=[{0}], EXPR$1=[SUM($1)], EXPR$2=[COUNT()], EXPR$3=[MIN($2)], EXPR$4=[MAX($1)])
-      ProjectRel(ENAME=[$1], EMPNO=[$0], DEPTNO=[$7])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
-    AggregateRel(group=[{0}], EXPR$1=[SUM($1)], EXPR$2=[COUNT()], EXPR$3=[MIN($2)], EXPR$4=[MAX($1)])
-      ProjectRel(ENAME=[$1], EMPNO=[$0], DEPTNO=[$7])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalAggregate(group=[{0}], EXPR$1=[SUM($1)], EXPR$2=[$SUM0($2)], EXPR$3=[MIN($3)], EXPR$4=[MAX($4)])
+  LogicalUnion(all=[true])
+    LogicalAggregate(group=[{0}], EXPR$1=[SUM($1)], EXPR$2=[COUNT()], EXPR$3=[MIN($2)], EXPR$4=[MAX($1)])
+      LogicalProject(ENAME=[$1], EMPNO=[$0], DEPTNO=[$7])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalAggregate(group=[{0}], EXPR$1=[SUM($1)], EXPR$2=[COUNT()], EXPR$3=[MIN($2)], EXPR$4=[MAX($1)])
+      LogicalProject(ENAME=[$1], EMPNO=[$0], DEPTNO=[$7])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -1171,19 +1171,19 @@ AggregateRel(group=[{0}], EXPR$1=[SUM($1)], EXPR$2=[$SUM0($2)], EXPR$3=[MIN($3)]
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-ProjectRel(DEPTNO=[$0], EXPR$1=[$2])
-  AggregateRel(group=[{0, 1}], EXPR$1=[MAX($2)])
-    ProjectRel(DEPTNO=[$7], FOUR=[$9], MGR=[$3])
-      ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], FOUR=[4], TWO_PLUS_THREE=[+(2, 3)], DEPTNO42=[+($7, 42)])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(DEPTNO=[$0], EXPR$1=[$2])
+  LogicalAggregate(group=[{0, 1}], EXPR$1=[MAX($2)])
+    LogicalProject(DEPTNO=[$7], FOUR=[$9], MGR=[$3])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], FOUR=[4], TWO_PLUS_THREE=[+(2, 3)], DEPTNO42=[+($7, 42)])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-ProjectRel(DEPTNO=[$0], EXPR$1=[$1])
-  AggregateRel(group=[{0}], EXPR$1=[MAX($2)])
-    ProjectRel(DEPTNO=[$7], FOUR=[4], MGR=[$3])
-      TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(DEPTNO=[$0], EXPR$1=[$1])
+  LogicalAggregate(group=[{0}], EXPR$1=[MAX($2)])
+    LogicalProject(DEPTNO=[$7], FOUR=[4], MGR=[$3])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -1195,19 +1195,19 @@ ProjectRel(DEPTNO=[$0], EXPR$1=[$1])
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-ProjectRel(DEPTNO=[$0], EXPR$1=[$2])
-  AggregateRel(group=[{0, 1}], EXPR$1=[MAX($2)])
-    ProjectRel(DEPTNO=[$7], FOUR=[$9], ENAME=[$1])
-      ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], FOUR=[4], TWO_PLUS_THREE=[+(2, 3)], DEPTNO42=[+($7, 42)])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(DEPTNO=[$0], EXPR$1=[$2])
+  LogicalAggregate(group=[{0, 1}], EXPR$1=[MAX($2)])
+    LogicalProject(DEPTNO=[$7], FOUR=[$9], ENAME=[$1])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], FOUR=[4], TWO_PLUS_THREE=[+(2, 3)], DEPTNO42=[+($7, 42)])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-ProjectRel(DEPTNO=[$0], EXPR$1=[$1])
-  AggregateRel(group=[{0}], EXPR$1=[MAX($2)])
-    ProjectRel(DEPTNO=[$7], FOUR=[4], ENAME=[$1])
-      TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(DEPTNO=[$0], EXPR$1=[$1])
+  LogicalAggregate(group=[{0}], EXPR$1=[MAX($2)])
+    LogicalProject(DEPTNO=[$7], FOUR=[4], ENAME=[$1])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -1219,19 +1219,19 @@ ProjectRel(DEPTNO=[$0], EXPR$1=[$1])
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-ProjectRel(DEPTNO=[$0], EXPR$1=[$4])
-  AggregateRel(group=[{0, 1, 2, 3}], EXPR$1=[MAX($4)])
-    ProjectRel(DEPTNO=[$7], FOUR=[$9], TWO_PLUS_THREE=[$10], DEPTNO42=[$11], MGR=[$3])
-      ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], FOUR=[4], TWO_PLUS_THREE=[+(2, 3)], DEPTNO42=[+($7, 42)])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(DEPTNO=[$0], EXPR$1=[$4])
+  LogicalAggregate(group=[{0, 1, 2, 3}], EXPR$1=[MAX($4)])
+    LogicalProject(DEPTNO=[$7], FOUR=[$9], TWO_PLUS_THREE=[$10], DEPTNO42=[$11], MGR=[$3])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], FOUR=[4], TWO_PLUS_THREE=[+(2, 3)], DEPTNO42=[+($7, 42)])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-ProjectRel(DEPTNO=[$0], EXPR$1=[$2])
-  AggregateRel(group=[{0, 1}], EXPR$1=[MAX($4)])
-    ProjectRel(DEPTNO=[$7], DEPTNO42=[+($7, 42)], FOUR=[4], TWO_PLUS_THREE=[+(2, 3)], MGR=[$3])
-      TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(DEPTNO=[$0], EXPR$1=[$2])
+  LogicalAggregate(group=[{0, 1}], EXPR$1=[MAX($4)])
+    LogicalProject(DEPTNO=[$7], DEPTNO42=[+($7, 42)], FOUR=[4], TWO_PLUS_THREE=[+(2, 3)], MGR=[$3])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -1243,19 +1243,19 @@ ProjectRel(DEPTNO=[$0], EXPR$1=[$2])
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-ProjectRel(DEPTNO=[$1], EXPR$1=[$2])
-  AggregateRel(group=[{0, 1}], EXPR$1=[MAX($2)])
-    ProjectRel(FOUR=[$9], DEPTNO=[$7], MGR=[$3])
-      ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], FOUR=[4], TWO_PLUS_THREE=[+(2, 3)], DEPTNO42=[+($7, 42)])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(DEPTNO=[$1], EXPR$1=[$2])
+  LogicalAggregate(group=[{0, 1}], EXPR$1=[MAX($2)])
+    LogicalProject(FOUR=[$9], DEPTNO=[$7], MGR=[$3])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], FOUR=[4], TWO_PLUS_THREE=[+(2, 3)], DEPTNO42=[+($7, 42)])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-ProjectRel(DEPTNO=[$0], EXPR$1=[$1])
-  AggregateRel(group=[{0}], EXPR$1=[MAX($2)])
-    ProjectRel(DEPTNO=[$7], FOUR=[4], MGR=[$3])
-      TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(DEPTNO=[$0], EXPR$1=[$1])
+  LogicalAggregate(group=[{0}], EXPR$1=[MAX($2)])
+    LogicalProject(DEPTNO=[$7], FOUR=[4], MGR=[$3])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -1267,19 +1267,19 @@ ProjectRel(DEPTNO=[$0], EXPR$1=[$1])
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-ProjectRel(DEPTNO=[$1], EXPR$1=[$2])
-  AggregateRel(group=[{0, 1}], EXPR$1=[MAX($2)])
-    ProjectRel($f0=[+(42, 24)], DEPTNO=[$7], MGR=[$3])
-      ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], FOUR=[4], TWO_PLUS_THREE=[+(2, 3)], DEPTNO42=[+($7, 42)])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(DEPTNO=[$1], EXPR$1=[$2])
+  LogicalAggregate(group=[{0, 1}], EXPR$1=[MAX($2)])
+    LogicalProject($f0=[+(42, 24)], DEPTNO=[$7], MGR=[$3])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], FOUR=[4], TWO_PLUS_THREE=[+(2, 3)], DEPTNO42=[+($7, 42)])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-ProjectRel(DEPTNO=[$0], EXPR$1=[$1])
-  AggregateRel(group=[{0}], EXPR$1=[MAX($2)])
-    ProjectRel(DEPTNO=[$7], $f0=[+(42, 24)], MGR=[$3])
-      TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(DEPTNO=[$0], EXPR$1=[$1])
+  LogicalAggregate(group=[{0}], EXPR$1=[MAX($2)])
+    LogicalProject(DEPTNO=[$7], $f0=[+(42, 24)], MGR=[$3])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -1291,18 +1291,18 @@ ProjectRel(DEPTNO=[$0], EXPR$1=[$1])
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-AggregateRel(group=[{0, 1}], EXPR$2=[MAX($2)])
-  ProjectRel(EXPR$0=[4], EXPR$1=[+(2, 3)], MGR=[$3])
-    ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], FOUR=[4], TWO_PLUS_THREE=[+(2, 3)], DEPTNO42=[+($7, 42)])
-      TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalAggregate(group=[{0, 1}], EXPR$2=[MAX($2)])
+  LogicalProject(EXPR$0=[4], EXPR$1=[+(2, 3)], MGR=[$3])
+    LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], FOUR=[4], TWO_PLUS_THREE=[+(2, 3)], DEPTNO42=[+($7, 42)])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-ProjectRel(EXPR$0=[$0], EXPR$1=[+(2, 3)], EXPR$2=[$1])
-  AggregateRel(group=[{0}], EXPR$2=[MAX($2)])
-    ProjectRel(EXPR$0=[4], EXPR$1=[+(2, 3)], MGR=[$3])
-      TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(EXPR$0=[$0], EXPR$1=[+(2, 3)], EXPR$2=[$1])
+  LogicalAggregate(group=[{0}], EXPR$2=[MAX($2)])
+    LogicalProject(EXPR$0=[4], EXPR$1=[+(2, 3)], MGR=[$3])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -1314,18 +1314,18 @@ ProjectRel(EXPR$0=[$0], EXPR$1=[+(2, 3)], EXPR$2=[$1])
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-AggregateRel(group=[{0, 1}], EXPR$2=[MAX($2)])
-  ProjectRel(EXPR$0=[4], EXPR$1=[+(2, 3)], FIVE=[$11])
-    ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], FOUR=[4], TWO_PLUS_THREE=[+(2, 3)], FIVE=[5], DEPTNO42=[+($7, 42)])
-      TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalAggregate(group=[{0, 1}], EXPR$2=[MAX($2)])
+  LogicalProject(EXPR$0=[4], EXPR$1=[+(2, 3)], FIVE=[$11])
+    LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], FOUR=[4], TWO_PLUS_THREE=[+(2, 3)], FIVE=[5], DEPTNO42=[+($7, 42)])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-ProjectRel(EXPR$0=[$0], EXPR$1=[+(2, 3)], EXPR$2=[$1])
-  AggregateRel(group=[{0}], EXPR$2=[MAX($2)])
-    ProjectRel(EXPR$0=[4], EXPR$1=[+(2, 3)], FIVE=[5])
-      TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(EXPR$0=[$0], EXPR$1=[+(2, 3)], EXPR$2=[$1])
+  LogicalAggregate(group=[{0}], EXPR$2=[MAX($2)])
+    LogicalProject(EXPR$0=[4], EXPR$1=[+(2, 3)], FIVE=[5])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -1337,18 +1337,18 @@ ProjectRel(EXPR$0=[$0], EXPR$1=[+(2, 3)], EXPR$2=[$1])
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-AggregateRel(group=[{0, 1}], EXPR$2=[MAX($2)])
-  ProjectRel(EXPR$0=[4], EXPR$1=[+(2, 3)], $f2=[5])
-    ProjectRel(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], FOUR=[4], TWO_PLUS_THREE=[+(2, 3)], FIVE=[5], DEPTNO42=[+($7, 42)])
-      TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalAggregate(group=[{0, 1}], EXPR$2=[MAX($2)])
+  LogicalProject(EXPR$0=[4], EXPR$1=[+(2, 3)], $f2=[5])
+    LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], FOUR=[4], TWO_PLUS_THREE=[+(2, 3)], FIVE=[5], DEPTNO42=[+($7, 42)])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-ProjectRel(EXPR$0=[$0], EXPR$1=[+(2, 3)], EXPR$2=[$1])
-  AggregateRel(group=[{0}], EXPR$2=[MAX($2)])
-    ProjectRel(EXPR$0=[4], EXPR$1=[+(2, 3)], $f2=[5])
-      TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(EXPR$0=[$0], EXPR$1=[+(2, 3)], EXPR$2=[$1])
+  LogicalAggregate(group=[{0}], EXPR$2=[MAX($2)])
+    LogicalProject(EXPR$0=[4], EXPR$1=[+(2, 3)], $f2=[5])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -1358,22 +1358,22 @@ ProjectRel(EXPR$0=[$0], EXPR$1=[+(2, 3)], EXPR$2=[$1])
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-ProjectRel(ENAME=[$1])
-  FilterRel(condition=[AND(=($7, $9), =($9, $18))])
-    JoinRel(condition=[true], joinType=[inner])
-      JoinRel(condition=[true], joinType=[inner])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
-        TableAccessRel(table=[[CATALOG, SALES, DEPT]])
-      TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(ENAME=[$1])
+  LogicalFilter(condition=[AND(=($7, $9), =($9, $18))])
+    LogicalJoin(condition=[true], joinType=[inner])
+      LogicalJoin(condition=[true], joinType=[inner])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+        LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-ProjectRel(ENAME=[$1])
-  MultiJoinRel(joinFilter=[AND(=($9, $18), =($7, $9))], isFullOuterJoin=[false], joinTypes=[[INNER, INNER, INNER]], outerJoinConditions=[[NULL, NULL, NULL]], projFields=[[ALL, ALL, ALL]])
-    TableAccessRel(table=[[CATALOG, SALES, EMP]])
-    TableAccessRel(table=[[CATALOG, SALES, DEPT]])
-    TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(ENAME=[$1])
+  MultiJoin(joinFilter=[AND(=($9, $18), =($7, $9))], isFullOuterJoin=[false], joinTypes=[[INNER, INNER, INNER]], outerJoinConditions=[[NULL, NULL, NULL]], projFields=[[ALL, ALL, ALL]])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -1383,20 +1383,20 @@ ProjectRel(ENAME=[$1])
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-ProjectRel(ENAME=[$1])
-  FilterRel(condition=[AND(=($7, $9), =($1, 'foo'))])
-    JoinRel(condition=[true], joinType=[inner])
-      TableAccessRel(table=[[CATALOG, SALES, EMP]])
-      TableAccessRel(table=[[CATALOG, SALES, DEPT]])
+LogicalProject(ENAME=[$1])
+  LogicalFilter(condition=[AND(=($7, $9), =($1, 'foo'))])
+    LogicalJoin(condition=[true], joinType=[inner])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+      LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-ProjectRel(ENAME=[$1])
-  JoinRel(condition=[=($7, $9)], joinType=[inner], semiJoinDone=[true])
-    FilterRel(condition=[=($1, 'foo')])
-      TableAccessRel(table=[[CATALOG, SALES, EMP]])
-    TableAccessRel(table=[[CATALOG, SALES, DEPT]])
+LogicalProject(ENAME=[$1])
+  LogicalJoin(condition=[=($7, $9)], joinType=[inner], semiJoinDone=[true])
+    LogicalFilter(condition=[=($1, 'foo')])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
         </Resource>
     </TestCase>
@@ -1406,23 +1406,23 @@ ProjectRel(ENAME=[$1])
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-ProjectRel(ENAME=[$1])
-  FilterRel(condition=[AND(=($7, $9), =($9, $18))])
-    JoinRel(condition=[true], joinType=[inner])
-      JoinRel(condition=[true], joinType=[inner])
-        TableAccessRel(table=[[CATALOG, SALES, EMP]])
-        TableAccessRel(table=[[CATALOG, SALES, DEPT]])
-      TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(ENAME=[$1])
+  LogicalFilter(condition=[AND(=($7, $9), =($9, $18))])
+    LogicalJoin(condition=[true], joinType=[inner])
+      LogicalJoin(condition=[true], joinType=[inner])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+        LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-ProjectRel(ENAME=[$1])
-  JoinRel(condition=[=($9, $18)], joinType=[inner], semiJoinDone=[true])
-    JoinRel(condition=[=($7, $9)], joinType=[inner], semiJoinDone=[true])
-      TableAccessRel(table=[[CATALOG, SALES, EMP]])
-      TableAccessRel(table=[[CATALOG, SALES, DEPT]])
-    TableAccessRel(table=[[CATALOG, SALES, EMP]])
+LogicalProject(ENAME=[$1])
+  LogicalJoin(condition=[=($9, $18)], joinType=[inner], semiJoinDone=[true])
+    LogicalJoin(condition=[=($7, $9)], joinType=[inner], semiJoinDone=[true])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+      LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -1446,56 +1446,56 @@ ProjectRel(DNAME=[$10], ENAME=[$1])
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-ProjectRel(A=[$0], B=[$1], C=[$2], D=[$3], E=[$4], F=[$5], G=[$6], H=[$7], I=[$8], J=[$9])
-  JoinRel(condition=[AND(=($0, $8), =($7, $9))], joinType=[inner])
-    ProjectRel(A=[$0], B=[$1], C=[$2], D=[$3], E=[$4], F=[$5], G=[$6], H=[$7])
-      JoinRel(condition=[AND(=($0, $4), =($1, $5), =($2, $6), =($3, $7))], joinType=[right])
-        ProjectRel(A=[$0], B=[$1], C=[$2], D=[$3])
-          JoinRel(condition=[AND(=($0, $2), =($1, $3))], joinType=[left])
-            ProjectRel(A=[$0], B=[$1])
-              JoinRel(condition=[=($0, $1)], joinType=[right])
-                TableAccessRel(table=[[CATALOG, SALES, A]])
-                TableAccessRel(table=[[CATALOG, SALES, B]])
-            ProjectRel(C=[$0], D=[$1])
-              JoinRel(condition=[=($0, $1)], joinType=[full])
-                TableAccessRel(table=[[CATALOG, SALES, C]])
-                TableAccessRel(table=[[CATALOG, SALES, D]])
-        ProjectRel(E=[$0], F=[$1], G=[$2], H=[$3])
-          JoinRel(condition=[AND(=($0, $2), =($1, $3))], joinType=[right])
-            ProjectRel(E=[$0], F=[$1])
-              JoinRel(condition=[=($0, $1)], joinType=[full])
-                TableAccessRel(table=[[CATALOG, SALES, E]])
-                TableAccessRel(table=[[CATALOG, SALES, F]])
-            ProjectRel(G=[$0], H=[$1])
-              JoinRel(condition=[=($0, $1)], joinType=[left])
-                TableAccessRel(table=[[CATALOG, SALES, G]])
-                TableAccessRel(table=[[CATALOG, SALES, H]])
-    ProjectRel(I=[$0], J=[$1])
-      JoinRel(condition=[=($0, $1)], joinType=[inner])
-        TableAccessRel(table=[[CATALOG, SALES, I]])
-        TableAccessRel(table=[[CATALOG, SALES, J]])
-]]>
-        </Resource>
-        <Resource name="planAfter">
-            <![CDATA[
-MultiJoinRel(joinFilter=[AND(=($0, $8), =($7, $9), =($8, $9))], isFullOuterJoin=[false], joinTypes=[[INNER, INNER, INNER]], outerJoinConditions=[[NULL, NULL, NULL]], projFields=[[ALL, ALL, ALL]])
-  MultiJoinRel(joinFilter=[true], isFullOuterJoin=[false], joinTypes=[[RIGHT, INNER]], outerJoinConditions=[[AND(=($0, $4), =($1, $5), =($2, $6), =($3, $7)), NULL]], projFields=[[ALL, ALL]])
-    MultiJoinRel(joinFilter=[true], isFullOuterJoin=[false], joinTypes=[[INNER, LEFT]], outerJoinConditions=[[NULL, AND(=($0, $2), =($1, $3))]], projFields=[[ALL, ALL]])
-      MultiJoinRel(joinFilter=[true], isFullOuterJoin=[false], joinTypes=[[RIGHT, INNER]], outerJoinConditions=[[=($0, $1), NULL]], projFields=[[ALL, ALL]])
-        TableAccessRel(table=[[CATALOG, SALES, A]])
-        TableAccessRel(table=[[CATALOG, SALES, B]])
-      MultiJoinRel(joinFilter=[=($0, $1)], isFullOuterJoin=[true], joinTypes=[[INNER, INNER]], outerJoinConditions=[[NULL, NULL]], projFields=[[ALL, ALL]])
-        TableAccessRel(table=[[CATALOG, SALES, C]])
-        TableAccessRel(table=[[CATALOG, SALES, D]])
-    MultiJoinRel(joinFilter=[true], isFullOuterJoin=[false], joinTypes=[[RIGHT, INNER]], outerJoinConditions=[[AND(=($0, $2), =($1, $3)), NULL]], projFields=[[ALL, ALL]])
-      MultiJoinRel(joinFilter=[=($0, $1)], isFullOuterJoin=[true], joinTypes=[[INNER, INNER]], outerJoinConditions=[[NULL, NULL]], projFields=[[ALL, ALL]])
-        TableAccessRel(table=[[CATALOG, SALES, E]])
-        TableAccessRel(table=[[CATALOG, SALES, F]])
-      MultiJoinRel(joinFilter=[true], isFullOuterJoin=[false], joinTypes=[[INNER, LEFT]], outerJoinConditions=[[NULL, =($0, $1)]], projFields=[[ALL, ALL]])
-        TableAccessRel(table=[[CATALOG, SALES, G]])
-        TableAccessRel(table=[[CATALOG, SALES, H]])
-  TableAccessRel(table=[[CATALOG, SALES, I]])
-  TableAccessRel(table=[[CATALOG, SALES, J]])
+LogicalProject(A=[$0], B=[$1], C=[$2], D=[$3], E=[$4], F=[$5], G=[$6], H=[$7], I=[$8], J=[$9])
+  LogicalJoin(condition=[AND(=($0, $8), =($7, $9))], joinType=[inner])
+    LogicalProject(A=[$0], B=[$1], C=[$2], D=[$3], E=[$4], F=[$

<TRUNCATED>

[23/58] [abbrv] [partial] incubator-calcite git commit: [CALCITE-306] Standardize code style for "import package.*; "

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/validate/SqlMonikerImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlMonikerImpl.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlMonikerImpl.java
index c4b560d..c7eebd8 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlMonikerImpl.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlMonikerImpl.java
@@ -14,16 +14,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.validate;
+package org.apache.calcite.sql.validate;
 
-import java.util.List;
-
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.util.Util;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
 
+import java.util.List;
+
 /**
  * A generic implementation of {@link SqlMoniker}.
  */

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/validate/SqlMonikerType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlMonikerType.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlMonikerType.java
index 6487992..4b64859 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlMonikerType.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlMonikerType.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.validate;
+package org.apache.calcite.sql.validate;
 
 /**
  * An enumeration of moniker types.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/validate/SqlMonotonicity.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlMonotonicity.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlMonotonicity.java
index 0d78517..778b38d 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlMonotonicity.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlMonotonicity.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.validate;
+package org.apache.calcite.sql.validate;
 
 /**
  * Enumeration of types of monotonicity.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/validate/SqlScopedShuttle.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlScopedShuttle.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlScopedShuttle.java
index 6af94b6..eca367b 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlScopedShuttle.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlScopedShuttle.java
@@ -14,12 +14,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.validate;
+package org.apache.calcite.sql.validate;
 
-import java.util.*;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.util.SqlShuttle;
+import org.apache.calcite.sql.util.SqlVisitor;
 
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.util.*;
+import java.util.Stack;
 
 /**
  * Refinement to {@link SqlShuttle} which maintains a stack of scopes.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedAggFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedAggFunction.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedAggFunction.java
index acc5836..6994e97 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedAggFunction.java
@@ -14,21 +14,26 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.validate;
+package org.apache.calcite.sql.validate;
 
-import java.util.List;
-
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.reltype.RelDataTypeFactory;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.util.Util;
-
-import net.hydromatic.optiq.*;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.schema.AggregateFunction;
+import org.apache.calcite.schema.FunctionParameter;
+import org.apache.calcite.sql.SqlAggFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.type.SqlOperandTypeChecker;
+import org.apache.calcite.sql.type.SqlOperandTypeInference;
+import org.apache.calcite.sql.type.SqlReturnTypeInference;
+import org.apache.calcite.util.Util;
 
 import com.google.common.base.Function;
 import com.google.common.collect.Lists;
 
+import java.util.List;
+
 /**
  * User-defined aggregate function.
  *

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedFunction.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedFunction.java
index fc9d771..49960f1 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedFunction.java
@@ -14,16 +14,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.validate;
+package org.apache.calcite.sql.validate;
 
-import java.util.List;
-
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.util.Util;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.schema.Function;
+import org.apache.calcite.sql.SqlFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.type.SqlOperandTypeChecker;
+import org.apache.calcite.sql.type.SqlOperandTypeInference;
+import org.apache.calcite.sql.type.SqlReturnTypeInference;
+import org.apache.calcite.util.Util;
 
-import net.hydromatic.optiq.Function;
+import java.util.List;
 
 /**
 * User-defined scalar function.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedTableFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedTableFunction.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedTableFunction.java
index 14af7dc..338a1ed 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedTableFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedTableFunction.java
@@ -14,25 +14,24 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.validate;
+package org.apache.calcite.sql.validate;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.schema.TableFunction;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.type.SqlOperandTypeChecker;
+import org.apache.calcite.sql.type.SqlOperandTypeInference;
+import org.apache.calcite.sql.type.SqlReturnTypeInference;
 
 import java.lang.reflect.Type;
 import java.util.List;
 
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.reltype.RelDataTypeFactory;
-import org.eigenbase.sql.SqlIdentifier;
-import org.eigenbase.sql.SqlNode;
-import org.eigenbase.sql.type.SqlOperandTypeChecker;
-import org.eigenbase.sql.type.SqlOperandTypeInference;
-import org.eigenbase.sql.type.SqlReturnTypeInference;
-
-import net.hydromatic.optiq.TableFunction;
-
 /**
  * User-defined table function.
- * <p>
- * Created by the validator, after resolving a function call to a function
+ *
+ * <p>Created by the validator, after resolving a function call to a function
  * defined in a Calcite schema.
 */
 public class SqlUserDefinedTableFunction extends SqlUserDefinedFunction {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedTableMacro.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedTableMacro.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedTableMacro.java
index 5ae9f02..665f363 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedTableMacro.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedTableMacro.java
@@ -14,35 +14,43 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.validate;
+package org.apache.calcite.sql.validate;
+
+import org.apache.calcite.adapter.enumerable.RexToLixTranslator;
+import org.apache.calcite.linq4j.tree.BlockBuilder;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.linq4j.tree.FunctionExpression;
+import org.apache.calcite.linq4j.tree.ParameterExpression;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeFactoryImpl;
+import org.apache.calcite.schema.Function;
+import org.apache.calcite.schema.FunctionParameter;
+import org.apache.calcite.schema.TableMacro;
+import org.apache.calcite.schema.TranslatableTable;
+import org.apache.calcite.sql.SqlFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlUtil;
+import org.apache.calcite.sql.type.SqlOperandTypeChecker;
+import org.apache.calcite.sql.type.SqlOperandTypeInference;
+import org.apache.calcite.sql.type.SqlReturnTypeInference;
+import org.apache.calcite.util.NlsString;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
 
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.reltype.RelDataTypeFactory;
-import org.eigenbase.reltype.RelDataTypeFactoryImpl;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.type.SqlOperandTypeChecker;
-import org.eigenbase.sql.type.SqlOperandTypeInference;
-import org.eigenbase.sql.type.SqlReturnTypeInference;
-import org.eigenbase.util.NlsString;
-import org.eigenbase.util.Pair;
-import org.eigenbase.util.Util;
-
-import net.hydromatic.linq4j.expressions.*;
-
-import net.hydromatic.optiq.Function;
-import net.hydromatic.optiq.FunctionParameter;
-import net.hydromatic.optiq.TableMacro;
-import net.hydromatic.optiq.TranslatableTable;
-import net.hydromatic.optiq.rules.java.RexToLixTranslator;
-
 /**
  * User-defined table macro.
- * <p>
- * Created by the validator, after resolving a function call to a function
+ *
+ * <p>Created by the validator, after resolving a function call to a function
  * defined in a Calcite schema.
 */
 public class SqlUserDefinedTableMacro extends SqlFunction {
@@ -68,8 +76,9 @@ public class SqlUserDefinedTableMacro extends SqlFunction {
   }
 
   /**
-   * Converts arguments from {@link org.eigenbase.sql.SqlNode} to java object
-   * format.
+   * Converts arguments from {@link org.apache.calcite.sql.SqlNode} to
+   * java object format.
+   *
    * @param typeFactory type factory used to convert the arguments
    * @param operandList input arguments
    * @param function target function to get parameter types from
@@ -94,10 +103,10 @@ public class SqlUserDefinedTableMacro extends SqlFunction {
       } else {
         arguments.add(null);
         if (failOnNonLiteral) {
-          throw new IllegalArgumentException(
-            "All arguments of call to macro " + opName + " should be "
-            + "literal. Actual argument #" + pair.left.getOrdinal() + " ("
-            + pair.left.getName() + ") is not literal: " + pair.right);
+          throw new IllegalArgumentException("All arguments of call to macro "
+              + opName + " should be literal. Actual argument #"
+              + pair.left.getOrdinal() + " (" + pair.left.getName()
+              + ") is not literal: " + pair.right);
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/validate/SqlValidator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidator.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidator.java
index 842c946..10f869d 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidator.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidator.java
@@ -14,32 +14,59 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.validate;
-
-import java.util.*;
-
-import org.eigenbase.reltype.*;
-import org.eigenbase.resource.Resources;
-import org.eigenbase.sql.*;
-import org.eigenbase.util.*;
+package org.apache.calcite.sql.validate;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.runtime.CalciteContextException;
+import org.apache.calcite.runtime.CalciteException;
+import org.apache.calcite.runtime.Resources;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlDataTypeSpec;
+import org.apache.calcite.sql.SqlDelete;
+import org.apache.calcite.sql.SqlDynamicParam;
+import org.apache.calcite.sql.SqlFunction;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlInsert;
+import org.apache.calcite.sql.SqlIntervalQualifier;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlMerge;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlNodeList;
+import org.apache.calcite.sql.SqlOperatorTable;
+import org.apache.calcite.sql.SqlSelect;
+import org.apache.calcite.sql.SqlUpdate;
+import org.apache.calcite.sql.SqlWindow;
+import org.apache.calcite.sql.SqlWith;
+import org.apache.calcite.sql.SqlWithItem;
+import org.apache.calcite.util.Util;
+
+import java.util.List;
+import java.util.Map;
 
 /**
  * Validates the parse tree of a SQL statement, and provides semantic
  * information about the parse tree.
  *
- * <p>To create an instance of the default validator implementation, call {@link
- * SqlValidatorUtil#newValidator}.
+ * <p>To create an instance of the default validator implementation, call
+ * {@link SqlValidatorUtil#newValidator}.
  *
  * <h2>Visitor pattern</h2>
  *
- * <p>The validator interface is an instance of the {@link
- * org.eigenbase.util.Glossary#VISITOR_PATTERN visitor pattern}. Implementations
+ * <p>The validator interface is an instance of the
+ * {@link org.apache.calcite.util.Glossary#VISITOR_PATTERN visitor pattern}.
+ * Implementations
  * of the {@link SqlNode#validate} method call the <code>validateXxx</code>
- * method appropriate to the kind of node: {@link
- * SqlLiteral#validate(SqlValidator, SqlValidatorScope)} calls {@link
- * #validateLiteral(org.eigenbase.sql.SqlLiteral)}; {@link
- * SqlCall#validate(SqlValidator, SqlValidatorScope)} calls {@link
- * #validateCall(SqlCall, SqlValidatorScope)}; and so forth.
+ * method appropriate to the kind of node:
+ * <ul>
+ * <li>{@link SqlLiteral#validate(SqlValidator, SqlValidatorScope)}
+ *     calls
+ *     {@link #validateLiteral(org.apache.calcite.sql.SqlLiteral)};
+ * <li>{@link SqlCall#validate(SqlValidator, SqlValidatorScope)}
+ *     calls
+ *     {@link #validateCall(SqlCall, SqlValidatorScope)};
+ * <li>and so forth.</ul>
  *
  * <p>The {@link SqlNode#validateExpr(SqlValidator, SqlValidatorScope)} method
  * is as {@link SqlNode#validate(SqlValidator, SqlValidatorScope)} but is called
@@ -48,27 +75,29 @@ import org.eigenbase.util.*;
  * <h2>Scopes and namespaces</h2>
  *
  * <p>In order to resolve names to objects, the validator builds a map of the
- * structure of the query. This map consists of two types of objects. A {@link
- * SqlValidatorScope} describes the tables and columns accessible at a
+ * structure of the query. This map consists of two types of objects. A
+ * {@link SqlValidatorScope} describes the tables and columns accessible at a
  * particular point in the query; and a {@link SqlValidatorNamespace} is a
  * description of a data source used in a query.
  *
  * <p>There are different kinds of namespace for different parts of the query.
- * for example {@link IdentifierNamespace} for table names, {@link
- * SelectNamespace} for SELECT queries, {@link SetopNamespace} for UNION, EXCEPT
+ * for example {@link IdentifierNamespace} for table names,
+ * {@link SelectNamespace} for SELECT queries,
+ * {@link SetopNamespace} for UNION, EXCEPT
  * and INTERSECT. A validator is allowed to wrap namespaces in other objects
  * which implement {@link SqlValidatorNamespace}, so don't try to cast your
- * namespace or use <code>instanceof</code>; use {@link
- * SqlValidatorNamespace#unwrap(Class)} and {@link
- * SqlValidatorNamespace#isWrapperFor(Class)} instead.</p>
+ * namespace or use <code>instanceof</code>; use
+ * {@link SqlValidatorNamespace#unwrap(Class)} and
+ * {@link SqlValidatorNamespace#isWrapperFor(Class)} instead.</p>
  *
  * <p>The validator builds the map by making a quick scan over the query when
  * the root {@link SqlNode} is first provided. Thereafter, it supplies the
  * correct scope or namespace object when it calls validation methods.</p>
  *
- * <p>The methods {@link #getSelectScope}, {@link #getFromScope}, {@link
- * #getWhereScope}, {@link #getGroupScope}, {@link #getHavingScope}, {@link
- * #getOrderScope} and {@link #getJoinScope} get the correct scope to resolve
+ * <p>The methods {@link #getSelectScope}, {@link #getFromScope},
+ * {@link #getWhereScope}, {@link #getGroupScope}, {@link #getHavingScope},
+ * {@link #getOrderScope} and {@link #getJoinScope} get the correct scope
+ * to resolve
  * names in a particular clause of a SQL statement.</p>
  */
 public interface SqlValidator {
@@ -225,8 +254,8 @@ public interface SqlValidator {
 
   /**
    * Validates the right-hand side of an OVER expression. It might be either
-   * an {@link SqlIdentifier identifier} referencing a window, or an {@link
-   * SqlWindow inline window specification}.
+   * an {@link SqlIdentifier identifier} referencing a window, or an
+   * {@link SqlWindow inline window specification}.
    *
    * @param windowOrId SqlNode that can be either SqlWindow with all the
    *                   components of a window spec or a SqlIdentifier with the
@@ -294,7 +323,7 @@ public interface SqlValidator {
    * @param e    The validation error
    * @return Exception containing positional information, never null
    */
-  EigenbaseContextException newValidationError(
+  CalciteContextException newValidationError(
       SqlNode node,
       Resources.ExInst<SqlValidatorException> e);
 
@@ -443,11 +472,14 @@ public interface SqlValidator {
    *
    * <ul>
    * <li>In FROM ({@link #getFromScope} , you can only see 'foo'.
+   *
    * <li>In WHERE ({@link #getWhereScope}), GROUP BY ({@link #getGroupScope}),
-   * SELECT ({@link #getSelectScope}), and the ON clause of the JOIN ({@link
-   * #getJoinScope}) you can see 'emp', 'dept', and 'foo'.
-   * <li>In ORDER BY ({@link #getOrderScope}), you can see the column alias
-   * 'x'; and tables 'emp', 'dept', and 'foo'.
+   * SELECT ({@link #getSelectScope}), and the ON clause of the JOIN
+   * ({@link #getJoinScope}) you can see 'emp', 'dept', and 'foo'.
+   *
+   * <li>In ORDER BY ({@link #getOrderScope}), you can see the column alias 'x';
+   * and tables 'emp', 'dept', and 'foo'.
+   *
    * </ul>
    *
    * @param select SELECT statement
@@ -457,8 +489,8 @@ public interface SqlValidator {
 
   /**
    * Returns the scope for resolving the SELECT, GROUP BY and HAVING clauses.
-   * Always a {@link SelectScope}; if this is an aggregation query, the {@link
-   * AggregatingScope} is stripped away.
+   * Always a {@link SelectScope}; if this is an aggregation query, the
+   * {@link AggregatingScope} is stripped away.
    *
    * @param select SELECT statement
    * @return naming scope for SELECT statement, sans any aggregating scope
@@ -602,7 +634,7 @@ public interface SqlValidator {
    *                           call
    * @param argTypes           Types of arguments
    */
-  EigenbaseException handleUnresolvedFunction(
+  CalciteException handleUnresolvedFunction(
       SqlCall call,
       SqlFunction unresolvedFunction,
       List<RelDataType> argTypes);

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorCatalogReader.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorCatalogReader.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorCatalogReader.java
index 378befc..915d9ec 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorCatalogReader.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorCatalogReader.java
@@ -14,12 +14,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.validate;
+package org.apache.calcite.sql.validate;
 
-import java.util.*;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.sql.SqlIdentifier;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
+import java.util.List;
 
 /**
  * Supplies catalog information for {@link SqlValidator}.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorException.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorException.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorException.java
index 919cf4f..fe739f2 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorException.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorException.java
@@ -14,29 +14,29 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.validate;
+package org.apache.calcite.sql.validate;
 
-import java.util.logging.*;
+import org.apache.calcite.util.CalciteValidatorException;
+
+import java.util.logging.Logger;
 
 // NOTE:  This class gets compiled independently of everything else so that
 // resource generation can use reflection.  That means it must have no
-// dependencies on other Eigenbase/Farrago code.
-
-import org.eigenbase.util14.*;
+// dependencies on other Calcite code.
 
 /**
  * Exception thrown while validating a SQL statement.
  *
- * <p>Unlike {@link org.eigenbase.util.EigenbaseException}, this is a checked
- * exception, which reminds code authors to wrap it in another exception
+ * <p>Unlike {@link org.apache.calcite.runtime.CalciteException}, this is a
+ * checked exception, which reminds code authors to wrap it in another exception
  * containing the line/column context.
  */
 public class SqlValidatorException extends Exception
-    implements EigenbaseValidatorException {
+    implements CalciteValidatorException {
   //~ Static fields/initializers ---------------------------------------------
 
   private static final Logger LOGGER =
-      Logger.getLogger("org.eigenbase.util.EigenbaseException");
+      Logger.getLogger("org.apache.calcite.runtime.CalciteException");
 
   static final long serialVersionUID = -831683113957131387L;
 
@@ -53,7 +53,7 @@ public class SqlValidatorException extends Exception
       Throwable cause) {
     super(message, cause);
 
-    // TODO: see note in EigenbaseException constructor
+    // TODO: see note in CalciteException constructor
     LOGGER.throwing("SqlValidatorException", "constructor", this);
     LOGGER.severe(toString());
   }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 631ccf6..5dcf320 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
@@ -14,32 +14,97 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.validate;
-
-import java.math.*;
-
-import java.util.*;
-import java.util.logging.*;
-
-import org.eigenbase.reltype.*;
-import org.eigenbase.resource.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.fun.*;
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.sql.util.*;
-import org.eigenbase.trace.*;
-import org.eigenbase.util.*;
-
-import net.hydromatic.linq4j.Linq4j;
-import net.hydromatic.linq4j.Ord;
+package org.apache.calcite.sql.validate;
+
+import org.apache.calcite.linq4j.Linq4j;
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.rel.type.RelRecordType;
+import org.apache.calcite.runtime.CalciteContextException;
+import org.apache.calcite.runtime.CalciteException;
+import org.apache.calcite.runtime.Feature;
+import org.apache.calcite.runtime.Resources;
+import org.apache.calcite.sql.JoinConditionType;
+import org.apache.calcite.sql.JoinType;
+import org.apache.calcite.sql.SqlAccessEnum;
+import org.apache.calcite.sql.SqlAccessType;
+import org.apache.calcite.sql.SqlBasicCall;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlCallBinding;
+import org.apache.calcite.sql.SqlDataTypeSpec;
+import org.apache.calcite.sql.SqlDelete;
+import org.apache.calcite.sql.SqlDynamicParam;
+import org.apache.calcite.sql.SqlExplain;
+import org.apache.calcite.sql.SqlFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlInsert;
+import org.apache.calcite.sql.SqlIntervalLiteral;
+import org.apache.calcite.sql.SqlIntervalQualifier;
+import org.apache.calcite.sql.SqlJoin;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlMerge;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlNodeList;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlOperatorTable;
+import org.apache.calcite.sql.SqlOrderBy;
+import org.apache.calcite.sql.SqlSampleSpec;
+import org.apache.calcite.sql.SqlSelect;
+import org.apache.calcite.sql.SqlSelectKeyword;
+import org.apache.calcite.sql.SqlSyntax;
+import org.apache.calcite.sql.SqlUnresolvedFunction;
+import org.apache.calcite.sql.SqlUpdate;
+import org.apache.calcite.sql.SqlUtil;
+import org.apache.calcite.sql.SqlWindow;
+import org.apache.calcite.sql.SqlWith;
+import org.apache.calcite.sql.SqlWithItem;
+import org.apache.calcite.sql.fun.SqlCase;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.AssignableOperandTypeChecker;
+import org.apache.calcite.sql.type.ReturnTypes;
+import org.apache.calcite.sql.type.SqlOperandTypeInference;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.type.SqlTypeUtil;
+import org.apache.calcite.sql.util.SqlShuttle;
+import org.apache.calcite.sql.util.SqlVisitor;
+import org.apache.calcite.util.BitString;
+import org.apache.calcite.util.Bug;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
+import org.apache.calcite.util.trace.CalciteTrace;
 
 import com.google.common.base.Function;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
-
-import static org.eigenbase.sql.SqlUtil.stripAs;
-import static org.eigenbase.util.Static.RESOURCE;
+import com.google.common.collect.Sets;
+
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.util.AbstractList;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Calendar;
+import java.util.Collections;
+import java.util.GregorianCalendar;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.IdentityHashMap;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.Stack;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import static org.apache.calcite.sql.SqlUtil.stripAs;
+import static org.apache.calcite.util.Static.RESOURCE;
 
 /**
  * Default implementation of {@link SqlValidator}.
@@ -47,7 +112,7 @@ import static org.eigenbase.util.Static.RESOURCE;
 public class SqlValidatorImpl implements SqlValidatorWithHints {
   //~ Static fields/initializers ---------------------------------------------
 
-  public static final Logger TRACER = EigenbaseTrace.PARSER_LOGGER;
+  public static final Logger TRACER = CalciteTrace.PARSER_LOGGER;
 
   /**
    * Alias generated for the source table when rewriting UPDATE to MERGE.
@@ -134,18 +199,19 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
       new IdentityHashMap<SqlSelect, SqlValidatorScope>();
 
   /**
-   * Maps a {@link SqlNode node} to the {@link SqlValidatorNamespace
-   * namespace} which describes what columns they contain.
+   * Maps a {@link SqlNode node} to the
+   * {@link SqlValidatorNamespace namespace} which describes what columns they
+   * contain.
    */
   protected final Map<SqlNode, SqlValidatorNamespace> namespaces =
       new IdentityHashMap<SqlNode, SqlValidatorNamespace>();
 
   /**
    * Set of select expressions used as cursor definitions. In standard SQL,
-   * only the top-level SELECT is a cursor; Eigenbase extends this with
+   * only the top-level SELECT is a cursor; Calcite extends this with
    * cursors as inputs to table functions.
    */
-  private final Set<SqlNode> cursorSet = new IdentityHashSet<SqlNode>();
+  private final Set<SqlNode> cursorSet = Sets.newIdentityHashSet();
 
   /**
    * Stack of objects that maintain information about function calls. A stack
@@ -1403,7 +1469,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
     return type;
   }
 
-  public EigenbaseException handleUnresolvedFunction(
+  public CalciteException handleUnresolvedFunction(
       SqlCall call,
       SqlFunction unresolvedFunction,
       List<RelDataType> argTypes) {
@@ -2498,7 +2564,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
         // ensure qualifier is good before attempting to validate literal
         validateIntervalQualifier(intervalQualifier);
         String intervalStr = interval.getIntervalLiteral();
-        // throws EigenbaseContextException if string is invalid
+        // throws CalciteContextException if string is invalid
         int[] values = intervalQualifier.evaluateIntervalLiteral(intervalStr,
             literal.getParserPosition(), typeFactory.getTypeSystem());
         Util.discard(values);
@@ -3544,7 +3610,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
   public void validateDynamicParam(SqlDynamicParam dynamicParam) {
   }
 
-  public EigenbaseContextException newValidationError(SqlNode node,
+  public CalciteContextException newValidationError(SqlNode node,
       Resources.ExInst<SqlValidatorException> e) {
     assert node != null;
     final SqlParserPos pos = node.getParserPosition();
@@ -3763,8 +3829,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
     final List<RelDataType> types = new ArrayList<RelDataType>();
     sqlQuery.accept(
         new SqlShuttle() {
-          @Override
-          public SqlNode visit(SqlDynamicParam param) {
+          @Override public SqlNode visit(SqlDynamicParam param) {
             RelDataType type = getValidatedNodeType(param);
             types.add(type);
             return param;
@@ -3773,13 +3838,11 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
     return typeFactory.createStructType(
         types,
         new AbstractList<String>() {
-          @Override
-          public String get(int index) {
+          @Override public String get(int index) {
             return "?" + index;
           }
 
-          @Override
-          public int size() {
+          @Override public int size() {
             return types.size();
           }
         });
@@ -4075,7 +4138,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
                   literal, RESOURCE.orderByOrdinalOutOfRange());
             }
 
-            // SQL ordinals are 1-based, but SortRel's are 0-based
+            // SQL ordinals are 1-based, but Sort's are 0-based
             int ordinal = intValue - 1;
             return nthSelectItem(ordinal, literal.getParserPosition());
           }
@@ -4141,6 +4204,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
     }
   }
 
+  /** Information about an identifier in a particular scope. */
   protected static class IdInfo {
     public final SqlValidatorScope scope;
     public final SqlIdentifier id;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorNamespace.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorNamespace.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorNamespace.java
index 549664d..b95204e 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorNamespace.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorNamespace.java
@@ -14,13 +14,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.validate;
+package org.apache.calcite.sql.validate;
 
-import java.util.*;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.util.Pair;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.util.*;
+import java.util.List;
 
 /**
  * A namespace describes the relation returned by a section of a SQL query.
@@ -33,17 +33,17 @@ import org.eigenbase.util.*;
  * namespace contains the constituent columns) and a subquery (the namespace
  * contains the columns in the SELECT clause of the subquery).
  *
- * <p>These various kinds of namespace are implemented by classes {@link
- * IdentifierNamespace} for table names, {@link SelectNamespace} for SELECT
- * queries, {@link SetopNamespace} for UNION, EXCEPT and INTERSECT, and so
- * forth. But if you are looking at a SELECT query and call {@link
- * SqlValidator#getNamespace(org.eigenbase.sql.SqlNode)}, you may not get a
- * SelectNamespace. Why? Because the validator is allowed to wrap namespaces in
- * other objects which implement {@link SqlValidatorNamespace}. Your
- * SelectNamespace will be there somewhere, but might be one or two levels deep.
- * Don't try to cast the namespace or use <code>instanceof</code>; use {@link
- * SqlValidatorNamespace#unwrap(Class)} and {@link
- * SqlValidatorNamespace#isWrapperFor(Class)} instead.</p>
+ * <p>These various kinds of namespace are implemented by classes
+ * {@link IdentifierNamespace} for table names, {@link SelectNamespace} for
+ * SELECT queries, {@link SetopNamespace} for UNION, EXCEPT and INTERSECT, and
+ * so forth. But if you are looking at a SELECT query and call
+ * {@link SqlValidator#getNamespace(org.apache.calcite.sql.SqlNode)}, you may
+ * not get a SelectNamespace. Why? Because the validator is allowed to wrap
+ * namespaces in other objects which implement
+ * {@link SqlValidatorNamespace}. Your SelectNamespace will be there somewhere,
+ * but might be one or two levels deep.  Don't try to cast the namespace or use
+ * <code>instanceof</code>; use {@link SqlValidatorNamespace#unwrap(Class)} and
+ * {@link SqlValidatorNamespace#isWrapperFor(Class)} instead.</p>
  *
  * @see SqlValidator
  * @see SqlValidatorScope
@@ -145,8 +145,8 @@ public interface SqlValidatorNamespace {
   /**
    * Returns a list of expressions which are monotonic in this namespace. For
    * example, if the namespace represents a relation ordered by a column
-   * called "TIMESTAMP", then the list would contain a {@link
-   * org.eigenbase.sql.SqlIdentifier} called "TIMESTAMP".
+   * called "TIMESTAMP", then the list would contain a
+   * {@link org.apache.calcite.sql.SqlIdentifier} called "TIMESTAMP".
    */
   List<Pair<SqlNode, SqlMonotonicity>> getMonotonicExprs();
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorScope.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorScope.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorScope.java
index 20a0dfb..db1e294 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorScope.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorScope.java
@@ -14,21 +14,26 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.validate;
+package org.apache.calcite.sql.validate;
 
-import java.util.*;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlNodeList;
+import org.apache.calcite.sql.SqlWindow;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
+import java.util.List;
 
 /**
  * Name-resolution scope. Represents any position in a parse tree than an
  * expression can be, or anything in the parse tree which has columns.
  *
- * <p>When validating an expression, say "foo"."bar", you first use the {@link
- * #resolve} method of the scope where the expression is defined to locate
- * "foo". If successful, this returns a {@link SqlValidatorNamespace namespace}
- * describing the type of the resulting object.
+ * <p>When validating an expression, say "foo"."bar", you first use the
+ * {@link #resolve} method of the scope where the expression is defined to
+ * locate "foo". If successful, this returns a
+ * {@link SqlValidatorNamespace namespace} describing the type of the resulting
+ * object.
  */
 public interface SqlValidatorScope {
   //~ Methods ----------------------------------------------------------------
@@ -59,11 +64,11 @@ public interface SqlValidatorScope {
    * Finds the table alias which is implicitly qualifying an unqualified
    * column name. Throws an error if there is not exactly one table.
    *
-   * <p>This method is only implemented in scopes (such as {@link
-   * org.eigenbase.sql.validate.SelectScope}) which can be the context for
-   * name-resolution. In scopes such as {@link
-   * org.eigenbase.sql.validate.IdentifierNamespace}, it throws {@link
-   * UnsupportedOperationException}.</p>
+   * <p>This method is only implemented in scopes (such as
+   * {@link org.apache.calcite.sql.validate.SelectScope}) which can be the
+   * context for name-resolution. In scopes such as
+   * {@link org.apache.calcite.sql.validate.IdentifierNamespace}, it throws
+   * {@link UnsupportedOperationException}.</p>
    *
    * @param columnName Column name
    * @param ctx        Validation context, to appear in any error thrown

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorTable.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorTable.java
index cfce480..7b2d48c 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorTable.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorTable.java
@@ -14,12 +14,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.validate;
+package org.apache.calcite.sql.validate;
 
-import java.util.List;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlAccessType;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
+import java.util.List;
 
 /**
  * Supplies a {@link SqlValidator} with the metadata for a table.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorUtil.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorUtil.java
index 3a9e919..304853a 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorUtil.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorUtil.java
@@ -14,20 +14,35 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.validate;
-
-import java.nio.charset.*;
-import java.util.*;
-
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.fun.*;
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.util.*;
-
-import net.hydromatic.optiq.prepare.Prepare;
+package org.apache.calcite.sql.validate;
+
+import org.apache.calcite.plan.RelOptSchemaWithSampling;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.prepare.Prepare;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlDataTypeSpec;
+import org.apache.calcite.sql.SqlDynamicParam;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlIntervalQualifier;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlNodeList;
+import org.apache.calcite.sql.SqlOperatorTable;
+import org.apache.calcite.sql.SqlUtil;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.SqlTypeUtil;
+import org.apache.calcite.util.Util;
+
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Set;
 
 /**
  * Utility methods related to validation.
@@ -101,11 +116,10 @@ public class SqlValidatorUtil {
         if (false) {
           // todo: enable this checking when we have a charset to
           //   collation mapping
-          throw new Error(
-              type.toString()
-                  + " was found to have charset '" + strCharset.name()
-                  + "' and a mismatched collation charset '"
-                  + colCharset.name() + "'");
+          throw new Error(type.toString()
+              + " was found to have charset '" + strCharset.name()
+              + "' and a mismatched collation charset '"
+              + colCharset.name() + "'");
         }
       }
     }
@@ -367,6 +381,8 @@ public class SqlValidatorUtil {
     }
   }
 
+  /** Suggests candidates for unique names, given the number of attempts so far
+   * and the number of expressions in the project list. */
   interface Suggester {
     String apply(String original, int attempt, int size);
   }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorWithHints.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorWithHints.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorWithHints.java
index d6fcc22..d131d94 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorWithHints.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorWithHints.java
@@ -14,12 +14,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.validate;
+package org.apache.calcite.sql.validate;
 
-import java.util.*;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.parser.SqlParserPos;
 
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.parser.*;
+import java.util.List;
 
 /**
  * Extends {@link SqlValidator} to allow discovery of useful data such as fully
@@ -32,7 +33,7 @@ public interface SqlValidatorWithHints extends SqlValidator {
   /**
    * Looks up completion hints for a syntatically correct SQL statement that
    * has been parsed into an expression tree. (Note this should be called
-   * after {@link #validate(org.eigenbase.sql.SqlNode)}.
+   * after {@link #validate(org.apache.calcite.sql.SqlNode)}.
    *
    * @param topNode top of expression tree in which to lookup completion hints
    * @param pos     indicates the position in the sql statement we want to get
@@ -54,8 +55,9 @@ public interface SqlValidatorWithHints extends SqlValidator {
    *
    * @param topNode top of expression tree in which to lookup the qualfied
    *                name for the SqlIdentifier
-   * @param pos     indicates the position of the {@link SqlIdentifier} in the sql
-   *                statement we want to get the qualified name for
+   * @param pos indicates the position of the {@link SqlIdentifier} in
+   *                the SQL statement we want to get the qualified
+   *                name for
    * @return a string of the fully qualified name of the {@link SqlIdentifier}
    * if the Parser position represents a valid {@link SqlIdentifier}. Else
    * return an empty string

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/validate/TableConstructorNamespace.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/TableConstructorNamespace.java b/core/src/main/java/org/apache/calcite/sql/validate/TableConstructorNamespace.java
index 061a608..05ddf53 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/TableConstructorNamespace.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/TableConstructorNamespace.java
@@ -14,12 +14,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.validate;
+package org.apache.calcite.sql.validate;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlInsert;
+import org.apache.calcite.sql.SqlNode;
 
-import static org.eigenbase.util.Static.RESOURCE;
+import static org.apache.calcite.util.Static.RESOURCE;
 
 /**
  * Namespace for a table constructor <code>VALUES (expr, expr, ...)</code>.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/validate/TableNamespace.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/TableNamespace.java b/core/src/main/java/org/apache/calcite/sql/validate/TableNamespace.java
index 80f11f1..1ec155e 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/TableNamespace.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/TableNamespace.java
@@ -14,10 +14,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.validate;
+package org.apache.calcite.sql.validate;
 
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.sql.SqlNode;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlNode;
 
 /** Namespace based on a table from the catalog. */
 class TableNamespace extends AbstractNamespace {
@@ -39,8 +39,7 @@ class TableNamespace extends AbstractNamespace {
     return null;
   }
 
-  @Override
-  public SqlValidatorTable getTable() {
+  @Override public SqlValidatorTable getTable() {
     return table;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/validate/UnnestNamespace.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/UnnestNamespace.java b/core/src/main/java/org/apache/calcite/sql/validate/UnnestNamespace.java
index 6a9897b..f5829f3 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/UnnestNamespace.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/UnnestNamespace.java
@@ -14,12 +14,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.validate;
+package org.apache.calcite.sql.validate;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.fun.*;
-import org.eigenbase.sql.type.*;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.type.MultisetSqlType;
 
 /**
  * Namespace for UNNEST.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/validate/WithItemNamespace.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/WithItemNamespace.java b/core/src/main/java/org/apache/calcite/sql/validate/WithItemNamespace.java
index 066551b..040f080 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/WithItemNamespace.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/WithItemNamespace.java
@@ -14,15 +14,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.validate;
+package org.apache.calcite.sql.validate;
 
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.reltype.RelDataTypeFactory;
-import org.eigenbase.reltype.RelDataTypeField;
-import org.eigenbase.sql.SqlIdentifier;
-import org.eigenbase.sql.SqlNode;
-import org.eigenbase.sql.SqlWithItem;
-import org.eigenbase.util.Pair;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlWithItem;
+import org.apache.calcite.util.Pair;
 
 /** Very similar to {@link AliasNamespace}. */
 class WithItemNamespace extends AbstractNamespace {
@@ -68,8 +68,7 @@ class WithItemNamespace extends AbstractNamespace {
       }
       ++i;
     }
-    throw new AssertionError(
-        "unknown field '" + name
+    throw new AssertionError("unknown field '" + name
         + "' in rowtype " + underlyingRowType);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/validate/WithNamespace.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/WithNamespace.java b/core/src/main/java/org/apache/calcite/sql/validate/WithNamespace.java
index 30c4d3b..d3cfb9a 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/WithNamespace.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/WithNamespace.java
@@ -14,11 +14,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.validate;
+package org.apache.calcite.sql.validate;
 
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.sql.*;
-import org.eigenbase.util.Util;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlWith;
+import org.apache.calcite.sql.SqlWithItem;
+import org.apache.calcite.util.Util;
 
 /**
  * Namespace for <code>WITH</code> clause.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/validate/WithScope.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/WithScope.java b/core/src/main/java/org/apache/calcite/sql/validate/WithScope.java
index 47a24ea..1866d7d 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/WithScope.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/WithScope.java
@@ -14,12 +14,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.validate;
+package org.apache.calcite.sql.validate;
 
-import java.util.List;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlWithItem;
 
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.SqlWithItem;
+import java.util.List;
 
 /** Scope providing the objects that are available after evaluating an item
  * in a WITH clause.
@@ -46,16 +46,14 @@ class WithScope extends ListScope {
     return withItem;
   }
 
-  @Override
-  public SqlValidatorNamespace getTableNamespace(List<String> names) {
+  @Override public SqlValidatorNamespace getTableNamespace(List<String> names) {
     if (names.size() == 1 && names.get(0).equals(withItem.name.getSimple())) {
       return validator.getNamespace(withItem);
     }
     return super.getTableNamespace(names);
   }
 
-  @Override
-  public SqlValidatorNamespace resolve(String name,
+  @Override public SqlValidatorNamespace resolve(String name,
       SqlValidatorScope[] ancestorOut,
       int[] offsetOut) {
     if (name.equals(withItem.name.getSimple())) {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/validate/package-info.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/package-info.java b/core/src/main/java/org/apache/calcite/sql/validate/package-info.java
index 12820a3..d904da3 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/package-info.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/package-info.java
@@ -18,6 +18,6 @@
 /**
  * SQL validation.
  */
-package org.eigenbase.sql.validate;
+package org.apache.calcite.sql.validate;
 
 // End package-info.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql2rel/DefaultValueFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql2rel/DefaultValueFactory.java b/core/src/main/java/org/apache/calcite/sql2rel/DefaultValueFactory.java
index 9d503b8..1e34826 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/DefaultValueFactory.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/DefaultValueFactory.java
@@ -14,14 +14,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql2rel;
+package org.apache.calcite.sql2rel;
 
-import java.util.List;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.SqlFunction;
 
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.rex.*;
-import org.eigenbase.sql.*;
+import java.util.List;
 
 /**
  * DefaultValueFactory supplies default values for INSERT, UPDATE, and NEW.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 73f6dc5..fcd3f4c 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/ReflectiveConvertletTable.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/ReflectiveConvertletTable.java
@@ -14,16 +14,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql2rel;
+package org.apache.calcite.sql2rel;
 
-import java.lang.reflect.*;
+import org.apache.calcite.rex.RexNode;
+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 java.util.*;
-
-import org.eigenbase.rex.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.util.*;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.util.HashMap;
+import java.util.Map;
 
 /**
  * Implementation of {@link SqlRexConvertletTable} which uses reflection to call
@@ -191,8 +195,8 @@ public class ReflectiveConvertletTable implements SqlRexConvertletTable {
   /**
    * Registers a convertlet for a given operator instance
    *
-   * @param op         Operator instance, say {@link
-   *                   org.eigenbase.sql.fun.SqlStdOperatorTable#MINUS}
+   * @param op         Operator instance, say
+   * {@link org.apache.calcite.sql.fun.SqlStdOperatorTable#MINUS}
    * @param convertlet Convertlet
    */
   protected void registerOp(SqlOperator op, SqlRexConvertlet convertlet) {


[18/58] [abbrv] [partial] incubator-calcite git commit: [CALCITE-306] Standardize code style for "import package.*; "

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java b/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
index 1ae9777..b6f6318 100644
--- a/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
+++ b/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
@@ -14,29 +14,35 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.parser;
-
-import java.util.*;
-
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.parser.impl.*;
-import org.eigenbase.sql.pretty.SqlPrettyWriter;
-import org.eigenbase.test.*;
-import org.eigenbase.util.*;
-import org.eigenbase.util14.*;
-
-import net.hydromatic.avatica.Casing;
-import net.hydromatic.avatica.Quoting;
+package org.apache.calcite.sql.parser;
+
+import org.apache.calcite.avatica.Casing;
+import org.apache.calcite.avatica.Quoting;
+import org.apache.calcite.sql.SqlDialect;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlSetOption;
+import org.apache.calcite.sql.parser.impl.SqlParserImpl;
+import org.apache.calcite.sql.pretty.SqlPrettyWriter;
+import org.apache.calcite.test.SqlValidatorTestCase;
+import org.apache.calcite.util.Bug;
+import org.apache.calcite.util.ConversionUtil;
+import org.apache.calcite.util.TestUtil;
+import org.apache.calcite.util.Util;
 
 import org.junit.Ignore;
 import org.junit.Test;
 
-import static org.hamcrest.CoreMatchers.*;
-import static org.junit.Assert.*;
+import java.util.Locale;
+
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
 
 /**
- * A <code>SqlParserTest</code> is a unit-test for {@link SqlParser the SQL
- * parser}.
+ * A <code>SqlParserTest</code> is a unit-test for
+ * {@link SqlParser the SQL parser}.
  */
 public class SqlParserTest {
   //~ Static fields/initializers ---------------------------------------------
@@ -126,13 +132,13 @@ public class SqlParserTest {
     checkFails(
         "select 0.5e1^.1^ from sales.emps",
         "(?s).*Encountered \".1\" at line 1, column 13.\n"
-        + "Was expecting one of:\n"
-        + "    \"FROM\" ...\n"
-        + "    \",\" ...\n"
-        + "    \"AS\" ...\n"
-        + "    <IDENTIFIER> ...\n"
-        + "    <QUOTED_IDENTIFIER> ...\n"
-        + ".*");
+            + "Was expecting one of:\n"
+            + "    \"FROM\" ...\n"
+            + "    \",\" ...\n"
+            + "    \"AS\" ...\n"
+            + "    <IDENTIFIER> ...\n"
+            + "    <QUOTED_IDENTIFIER> ...\n"
+            + ".*");
   }
 
   @Test public void testInvalidToken() {
@@ -147,16 +153,16 @@ public class SqlParserTest {
   @Test public void testDerivedColumnList() {
     check("select * from emp as e (empno, gender) where true",
         "SELECT *\n"
-        + "FROM `EMP` AS `E` (`EMPNO`, `GENDER`)\n"
-        + "WHERE TRUE");
+            + "FROM `EMP` AS `E` (`EMPNO`, `GENDER`)\n"
+            + "WHERE TRUE");
   }
 
   @Test public void testDerivedColumnListInJoin() {
     check(
         "select * from emp as e (empno, gender) join dept as d (deptno, dname) on emp.deptno = dept.deptno",
         "SELECT *\n"
-        + "FROM `EMP` AS `E` (`EMPNO`, `GENDER`)\n"
-        + "INNER JOIN `DEPT` AS `D` (`DEPTNO`, `DNAME`) ON (`EMP`.`DEPTNO` = `DEPT`.`DEPTNO`)");
+            + "FROM `EMP` AS `E` (`EMPNO`, `GENDER`)\n"
+            + "INNER JOIN `DEPT` AS `D` (`DEPTNO`, `DNAME`) ON (`EMP`.`DEPTNO` = `DEPT`.`DEPTNO`)");
   }
 
   @Ignore
@@ -179,14 +185,14 @@ public class SqlParserTest {
     check(
         "select 1 as foo from emp",
         "SELECT 1 AS `FOO`\n"
-        + "FROM `EMP`");
+            + "FROM `EMP`");
   }
 
   @Test public void testColumnAliasWithoutAs() {
     check(
         "select 1 foo from emp",
         "SELECT 1 AS `FOO`\n"
-        + "FROM `EMP`");
+            + "FROM `EMP`");
   }
 
   @Test public void testEmbeddedDate() {
@@ -207,33 +213,33 @@ public class SqlParserTest {
     check(
         "select not true, not false, not null, not unknown from t",
         "SELECT (NOT TRUE), (NOT FALSE), (NOT NULL), (NOT UNKNOWN)\n"
-        + "FROM `T`");
+            + "FROM `T`");
   }
 
   @Test public void testBooleanPrecedenceAndAssociativity() {
     check(
         "select * from t where true and false",
         "SELECT *\n"
-        + "FROM `T`\n"
-        + "WHERE (TRUE AND FALSE)");
+            + "FROM `T`\n"
+            + "WHERE (TRUE AND FALSE)");
 
     check(
         "select * from t where null or unknown and unknown",
         "SELECT *\n"
-        + "FROM `T`\n"
-        + "WHERE (NULL OR (UNKNOWN AND UNKNOWN))");
+            + "FROM `T`\n"
+            + "WHERE (NULL OR (UNKNOWN AND UNKNOWN))");
 
     check(
         "select * from t where true and (true or true) or false",
         "SELECT *\n"
-        + "FROM `T`\n"
-        + "WHERE ((TRUE AND (TRUE OR TRUE)) OR FALSE)");
+            + "FROM `T`\n"
+            + "WHERE ((TRUE AND (TRUE OR TRUE)) OR FALSE)");
 
     check(
         "select * from t where 1 and true",
         "SELECT *\n"
-        + "FROM `T`\n"
-        + "WHERE (1 AND TRUE)");
+            + "FROM `T`\n"
+            + "WHERE (1 AND TRUE)");
   }
 
   @Test public void testIsBooleans() {
@@ -243,14 +249,14 @@ public class SqlParserTest {
       check(
           "select * from t where nOt fAlSe Is " + inOut,
           "SELECT *\n"
-          + "FROM `T`\n"
-          + "WHERE ((NOT FALSE) IS " + inOut + ")");
+              + "FROM `T`\n"
+              + "WHERE ((NOT FALSE) IS " + inOut + ")");
 
       check(
           "select * from t where c1=1.1 IS NOT " + inOut,
           "SELECT *\n"
-          + "FROM `T`\n"
-          + "WHERE ((`C1` = 1.1) IS NOT " + inOut + ")");
+              + "FROM `T`\n"
+              + "WHERE ((`C1` = 1.1) IS NOT " + inOut + ")");
     }
   }
 
@@ -258,28 +264,28 @@ public class SqlParserTest {
     check(
         "select * from t where x is unknown is not unknown",
         "SELECT *\n"
-        + "FROM `T`\n"
-        + "WHERE ((`X` IS UNKNOWN) IS NOT UNKNOWN)");
+            + "FROM `T`\n"
+            + "WHERE ((`X` IS UNKNOWN) IS NOT UNKNOWN)");
 
     check(
         "select 1 from t where not true is unknown",
         "SELECT 1\n"
-        + "FROM `T`\n"
-        + "WHERE ((NOT TRUE) IS UNKNOWN)");
+            + "FROM `T`\n"
+            + "WHERE ((NOT TRUE) IS UNKNOWN)");
 
     check(
         "select * from t where x is unknown is not unknown is false is not false"
-        + " is true is not true is null is not null",
+            + " is true is not true is null is not null",
         "SELECT *\n"
-        + "FROM `T`\n"
-        + "WHERE ((((((((`X` IS UNKNOWN) IS NOT UNKNOWN) IS FALSE) IS NOT FALSE) IS TRUE) IS NOT TRUE) IS NULL) IS NOT NULL)");
+            + "FROM `T`\n"
+            + "WHERE ((((((((`X` IS UNKNOWN) IS NOT UNKNOWN) IS FALSE) IS NOT FALSE) IS TRUE) IS NOT TRUE) IS NULL) IS NOT NULL)");
 
     // combine IS postfix operators with infix (AND) and prefix (NOT) ops
     check(
         "select * from t where x is unknown is false and x is unknown is true or not y is unknown is not null",
         "SELECT *\n"
-        + "FROM `T`\n"
-        + "WHERE ((((`X` IS UNKNOWN) IS FALSE) AND ((`X` IS UNKNOWN) IS TRUE)) OR (((NOT `Y`) IS UNKNOWN) IS NOT NULL))");
+            + "FROM `T`\n"
+            + "WHERE ((((`X` IS UNKNOWN) IS FALSE) AND ((`X` IS UNKNOWN) IS TRUE)) OR (((NOT `Y`) IS UNKNOWN) IS NOT NULL))");
   }
 
   @Test public void testEqualNotEqual() {
@@ -307,44 +313,44 @@ public class SqlParserTest {
     check(
         "select * from t where price between 1 and 2",
         "SELECT *\n"
-        + "FROM `T`\n"
-        + "WHERE (`PRICE` BETWEEN ASYMMETRIC 1 AND 2)");
+            + "FROM `T`\n"
+            + "WHERE (`PRICE` BETWEEN ASYMMETRIC 1 AND 2)");
 
     check(
         "select * from t where price between symmetric 1 and 2",
         "SELECT *\n"
-        + "FROM `T`\n"
-        + "WHERE (`PRICE` BETWEEN SYMMETRIC 1 AND 2)");
+            + "FROM `T`\n"
+            + "WHERE (`PRICE` BETWEEN SYMMETRIC 1 AND 2)");
 
     check(
         "select * from t where price not between symmetric 1 and 2",
         "SELECT *\n"
-        + "FROM `T`\n"
-        + "WHERE (`PRICE` NOT BETWEEN SYMMETRIC 1 AND 2)");
+            + "FROM `T`\n"
+            + "WHERE (`PRICE` NOT BETWEEN SYMMETRIC 1 AND 2)");
 
     check(
         "select * from t where price between ASYMMETRIC 1 and 2+2*2",
         "SELECT *\n"
-        + "FROM `T`\n"
-        + "WHERE (`PRICE` BETWEEN ASYMMETRIC 1 AND (2 + (2 * 2)))");
+            + "FROM `T`\n"
+            + "WHERE (`PRICE` BETWEEN ASYMMETRIC 1 AND (2 + (2 * 2)))");
 
     check(
         "select * from t where price > 5 and price not between 1 + 2 and 3 * 4 AnD price is null",
         "SELECT *\n"
-        + "FROM `T`\n"
-        + "WHERE (((`PRICE` > 5) AND (`PRICE` NOT BETWEEN ASYMMETRIC (1 + 2) AND (3 * 4))) AND (`PRICE` IS NULL))");
+            + "FROM `T`\n"
+            + "WHERE (((`PRICE` > 5) AND (`PRICE` NOT BETWEEN ASYMMETRIC (1 + 2) AND (3 * 4))) AND (`PRICE` IS NULL))");
 
     check(
         "select * from t where price > 5 and price between 1 + 2 and 3 * 4 + price is null",
         "SELECT *\n"
-        + "FROM `T`\n"
-        + "WHERE ((`PRICE` > 5) AND ((`PRICE` BETWEEN ASYMMETRIC (1 + 2) AND ((3 * 4) + `PRICE`)) IS NULL))");
+            + "FROM `T`\n"
+            + "WHERE ((`PRICE` > 5) AND ((`PRICE` BETWEEN ASYMMETRIC (1 + 2) AND ((3 * 4) + `PRICE`)) IS NULL))");
 
     check(
         "select * from t where price > 5 and price between 1 + 2 and 3 * 4 or price is null",
         "SELECT *\n"
-        + "FROM `T`\n"
-        + "WHERE (((`PRICE` > 5) AND (`PRICE` BETWEEN ASYMMETRIC (1 + 2) AND (3 * 4))) OR (`PRICE` IS NULL))");
+            + "FROM `T`\n"
+            + "WHERE (((`PRICE` > 5) AND (`PRICE` BETWEEN ASYMMETRIC (1 + 2) AND (3 * 4))) OR (`PRICE` IS NULL))");
 
     check(
         "values a between c and d and e and f between g and h",
@@ -382,29 +388,29 @@ public class SqlParserTest {
     check(
         "select c1*1,c2  + 2,c3/3,c4-4,c5*c4  from t",
         "SELECT (`C1` * 1), (`C2` + 2), (`C3` / 3), (`C4` - 4), (`C5` * `C4`)\n"
-        + "FROM `T`");
+            + "FROM `T`");
   }
 
   @Test public void testRow() {
     check(
         "select t.r.\"EXPR$1\", t.r.\"EXPR$0\" from (select (1,2) r from sales.depts) t",
         "SELECT `T`.`R`.`EXPR$1`, `T`.`R`.`EXPR$0`\n"
-        + "FROM (SELECT (ROW(1, 2)) AS `R`\n"
-        + "FROM `SALES`.`DEPTS`) AS `T`");
+            + "FROM (SELECT (ROW(1, 2)) AS `R`\n"
+            + "FROM `SALES`.`DEPTS`) AS `T`");
 
     check(
         "select t.r.\"EXPR$1\".\"EXPR$2\" "
-        + "from (select ((1,2),(3,4,5)) r from sales.depts) t",
+            + "from (select ((1,2),(3,4,5)) r from sales.depts) t",
         "SELECT `T`.`R`.`EXPR$1`.`EXPR$2`\n"
-        + "FROM (SELECT (ROW((ROW(1, 2)), (ROW(3, 4, 5)))) AS `R`\n"
-        + "FROM `SALES`.`DEPTS`) AS `T`");
+            + "FROM (SELECT (ROW((ROW(1, 2)), (ROW(3, 4, 5)))) AS `R`\n"
+            + "FROM `SALES`.`DEPTS`) AS `T`");
 
     check(
         "select t.r.\"EXPR$1\".\"EXPR$2\" "
-        + "from (select ((1,2),(3,4,5,6)) r from sales.depts) t",
+            + "from (select ((1,2),(3,4,5,6)) r from sales.depts) t",
         "SELECT `T`.`R`.`EXPR$1`.`EXPR$2`\n"
-        + "FROM (SELECT (ROW((ROW(1, 2)), (ROW(3, 4, 5, 6)))) AS `R`\n"
-        + "FROM `SALES`.`DEPTS`) AS `T`");
+            + "FROM (SELECT (ROW((ROW(1, 2)), (ROW(3, 4, 5, 6)))) AS `R`\n"
+            + "FROM `SALES`.`DEPTS`) AS `T`");
   }
 
   @Test public void testOverlaps() {
@@ -437,44 +443,44 @@ public class SqlParserTest {
     check(
         "select x is distinct from y from t",
         "SELECT (`X` IS DISTINCT FROM `Y`)\n"
-        + "FROM `T`");
+            + "FROM `T`");
 
     check(
         "select * from t where x is distinct from y",
         "SELECT *\n"
-        + "FROM `T`\n"
-        + "WHERE (`X` IS DISTINCT FROM `Y`)");
+            + "FROM `T`\n"
+            + "WHERE (`X` IS DISTINCT FROM `Y`)");
 
     check(
         "select * from t where x is distinct from (4,5,6)",
         "SELECT *\n"
-        + "FROM `T`\n"
-        + "WHERE (`X` IS DISTINCT FROM (ROW(4, 5, 6)))");
+            + "FROM `T`\n"
+            + "WHERE (`X` IS DISTINCT FROM (ROW(4, 5, 6)))");
 
     check(
         "select * from t where true is distinct from true",
         "SELECT *\n"
-        + "FROM `T`\n"
-        + "WHERE (TRUE IS DISTINCT FROM TRUE)");
+            + "FROM `T`\n"
+            + "WHERE (TRUE IS DISTINCT FROM TRUE)");
 
     check(
         "select * from t where true is distinct from true is true",
         "SELECT *\n"
-        + "FROM `T`\n"
-        + "WHERE ((TRUE IS DISTINCT FROM TRUE) IS TRUE)");
+            + "FROM `T`\n"
+            + "WHERE ((TRUE IS DISTINCT FROM TRUE) IS TRUE)");
   }
 
   @Test public void testIsNotDistinct() {
     check(
         "select x is not distinct from y from t",
         "SELECT (`X` IS NOT DISTINCT FROM `Y`)\n"
-        + "FROM `T`");
+            + "FROM `T`");
 
     check(
         "select * from t where true is not distinct from true",
         "SELECT *\n"
-        + "FROM `T`\n"
-        + "WHERE (TRUE IS NOT DISTINCT FROM TRUE)");
+            + "FROM `T`\n"
+            + "WHERE (TRUE IS NOT DISTINCT FROM TRUE)");
   }
 
   @Test public void testCast() {
@@ -509,22 +515,22 @@ public class SqlParserTest {
     check(
         "select * from t where x like '%abc%'",
         "SELECT *\n"
-        + "FROM `T`\n"
-        + "WHERE (`X` LIKE '%abc%')");
+            + "FROM `T`\n"
+            + "WHERE (`X` LIKE '%abc%')");
 
     check(
         "select * from t where x+1 not siMilaR to '%abc%' ESCAPE 'e'",
         "SELECT *\n"
-        + "FROM `T`\n"
-        + "WHERE ((`X` + 1) NOT SIMILAR TO '%abc%' ESCAPE 'e')");
+            + "FROM `T`\n"
+            + "WHERE ((`X` + 1) NOT SIMILAR TO '%abc%' ESCAPE 'e')");
 
     // LIKE has higher precedence than AND
     check(
         "select * from t where price > 5 and x+2*2 like y*3+2 escape (select*from t)",
         "SELECT *\n"
-        + "FROM `T`\n"
-        + "WHERE ((`PRICE` > 5) AND ((`X` + (2 * 2)) LIKE ((`Y` * 3) + 2) ESCAPE (SELECT *\n"
-        + "FROM `T`)))");
+            + "FROM `T`\n"
+            + "WHERE ((`PRICE` > 5) AND ((`X` + (2 * 2)) LIKE ((`Y` * 3) + 2) ESCAPE (SELECT *\n"
+            + "FROM `T`)))");
 
     check(
         "values a and b like c",
@@ -586,22 +592,22 @@ public class SqlParserTest {
     check(
         "select * from t where x similar to '%abc%'",
         "SELECT *\n"
-        + "FROM `T`\n"
-        + "WHERE (`X` SIMILAR TO '%abc%')");
+            + "FROM `T`\n"
+            + "WHERE (`X` SIMILAR TO '%abc%')");
 
     check(
         "select * from t where x+1 not siMilaR to '%abc%' ESCAPE 'e'",
         "SELECT *\n"
-        + "FROM `T`\n"
-        + "WHERE ((`X` + 1) NOT SIMILAR TO '%abc%' ESCAPE 'e')");
+            + "FROM `T`\n"
+            + "WHERE ((`X` + 1) NOT SIMILAR TO '%abc%' ESCAPE 'e')");
 
     // SIMILAR TO has higher precedence than AND
     check(
         "select * from t where price > 5 and x+2*2 SIMILAR TO y*3+2 escape (select*from t)",
         "SELECT *\n"
-        + "FROM `T`\n"
-        + "WHERE ((`PRICE` > 5) AND ((`X` + (2 * 2)) SIMILAR TO ((`Y` * 3) + 2) ESCAPE (SELECT *\n"
-        + "FROM `T`)))");
+            + "FROM `T`\n"
+            + "WHERE ((`PRICE` > 5) AND ((`X` + (2 * 2)) SIMILAR TO ((`Y` * 3) + 2) ESCAPE (SELECT *\n"
+            + "FROM `T`)))");
 
     // Mixed LIKE and SIMILAR TO
     check(
@@ -612,8 +618,8 @@ public class SqlParserTest {
     check(
         "values a similar to (select * from t where a like b escape c) escape d",
         "(VALUES (ROW((`A` SIMILAR TO (SELECT *\n"
-        + "FROM `T`\n"
-        + "WHERE (`A` LIKE `B` ESCAPE `C`)) ESCAPE `D`))))");
+            + "FROM `T`\n"
+            + "WHERE (`A` LIKE `B` ESCAPE `C`)) ESCAPE `D`))))");
   }
 
   @Test public void testFoo() {
@@ -632,27 +638,27 @@ public class SqlParserTest {
     check(
         "select * from dept where exists (select 1 from emp where emp.deptno = dept.deptno)",
         "SELECT *\n"
-        + "FROM `DEPT`\n"
-        + "WHERE (EXISTS (SELECT 1\n"
-        + "FROM `EMP`\n"
-        + "WHERE (`EMP`.`DEPTNO` = `DEPT`.`DEPTNO`)))");
+            + "FROM `DEPT`\n"
+            + "WHERE (EXISTS (SELECT 1\n"
+            + "FROM `EMP`\n"
+            + "WHERE (`EMP`.`DEPTNO` = `DEPT`.`DEPTNO`)))");
   }
 
   @Test public void testExistsInWhere() {
     check(
         "select * from emp where 1 = 2 and exists (select 1 from dept) and 3 = 4",
         "SELECT *\n"
-        + "FROM `EMP`\n"
-        + "WHERE (((1 = 2) AND (EXISTS (SELECT 1\n"
-        + "FROM `DEPT`))) AND (3 = 4))");
+            + "FROM `EMP`\n"
+            + "WHERE (((1 = 2) AND (EXISTS (SELECT 1\n"
+            + "FROM `DEPT`))) AND (3 = 4))");
   }
 
   @Test public void testFromWithAs() {
     check(
         "select 1 from emp as e where 1",
         "SELECT 1\n"
-        + "FROM `EMP` AS `E`\n"
-        + "WHERE 1");
+            + "FROM `EMP` AS `E`\n"
+            + "WHERE 1");
   }
 
   @Test public void testConcat() {
@@ -681,7 +687,7 @@ public class SqlParserTest {
     check(
         "select substring('Eggs and ham', 1, 3 + 2) || ' benedict' from emp",
         "SELECT (SUBSTRING('Eggs and ham' FROM 1 FOR (3 + 2)) || ' benedict')\n"
-        + "FROM `EMP`");
+            + "FROM `EMP`");
     checkExp(
         "log10(1)\r\n+power(2, mod(\r\n3\n\t\t\f\n,ln(4))*log10(5)-6*log10(7/abs(8)+9))*power(10,11)",
         "(LOG10(1) + (POWER(2, ((MOD(3, LN(4)) * LOG10(5)) - (6 * LOG10(((7 / ABS(8)) + 9))))) * POWER(10, 11)))");
@@ -694,7 +700,7 @@ public class SqlParserTest {
     check(
         "select count(1), count(distinct 2) from emp",
         "SELECT COUNT(1), COUNT(DISTINCT 2)\n"
-        + "FROM `EMP`");
+            + "FROM `EMP`");
   }
 
   @Test public void testFunctionInFunction() {
@@ -705,24 +711,24 @@ public class SqlParserTest {
     check(
         "select deptno, min(foo) as x from emp group by deptno, gender",
         "SELECT `DEPTNO`, MIN(`FOO`) AS `X`\n"
-        + "FROM `EMP`\n"
-        + "GROUP BY `DEPTNO`, `GENDER`");
+            + "FROM `EMP`\n"
+            + "GROUP BY `DEPTNO`, `GENDER`");
   }
 
   @Test public void testGroupEmpty() {
     check(
         "select count(*) from emp group by ()",
         "SELECT COUNT(*)\n"
-        + "FROM `EMP`\n"
-        + "GROUP BY ()");
+            + "FROM `EMP`\n"
+            + "GROUP BY ()");
 
     check(
         "select count(*) from emp group by () having 1 = 2 order by 3",
         "SELECT COUNT(*)\n"
-        + "FROM `EMP`\n"
-        + "GROUP BY ()\n"
-        + "HAVING (1 = 2)\n"
-        + "ORDER BY 3");
+            + "FROM `EMP`\n"
+            + "GROUP BY ()\n"
+            + "HAVING (1 = 2)\n"
+            + "ORDER BY 3");
 
     checkFails(
         "select 1 from emp group by ()^,^ x",
@@ -736,18 +742,18 @@ public class SqlParserTest {
     check(
         "select 1 from emp group by (empno + deptno)",
         "SELECT 1\n"
-        + "FROM `EMP`\n"
-        + "GROUP BY (`EMPNO` + `DEPTNO`)");
+            + "FROM `EMP`\n"
+            + "GROUP BY (`EMPNO` + `DEPTNO`)");
   }
 
   @Test public void testHavingAfterGroup() {
     check(
         "select deptno from emp group by deptno, emp having count(*) > 5 and 1 = 2 order by 5, 2",
         "SELECT `DEPTNO`\n"
-        + "FROM `EMP`\n"
-        + "GROUP BY `DEPTNO`, `EMP`\n"
-        + "HAVING ((COUNT(*) > 5) AND (1 = 2))\n"
-        + "ORDER BY 5, 2");
+            + "FROM `EMP`\n"
+            + "GROUP BY `DEPTNO`, `EMP`\n"
+            + "HAVING ((COUNT(*) > 5) AND (1 = 2))\n"
+            + "ORDER BY 5, 2");
   }
 
   @Test public void testHavingBeforeGroupFails() {
@@ -760,79 +766,79 @@ public class SqlParserTest {
     check(
         "select deptno from emp having count(*) > 5",
         "SELECT `DEPTNO`\n"
-        + "FROM `EMP`\n"
-        + "HAVING (COUNT(*) > 5)");
+            + "FROM `EMP`\n"
+            + "HAVING (COUNT(*) > 5)");
   }
 
   @Test public void testWith() {
     check(
         "with femaleEmps as (select * from emps where gender = 'F')"
-        + "select deptno from femaleEmps",
+            + "select deptno from femaleEmps",
         "WITH `FEMALEEMPS` AS (SELECT *\n"
-        + "FROM `EMPS`\n"
-        + "WHERE (`GENDER` = 'F')) SELECT `DEPTNO`\n"
-        + "FROM `FEMALEEMPS`");
+            + "FROM `EMPS`\n"
+            + "WHERE (`GENDER` = 'F')) SELECT `DEPTNO`\n"
+            + "FROM `FEMALEEMPS`");
   }
 
   @Test public void testWith2() {
     check(
         "with femaleEmps as (select * from emps where gender = 'F'),\n"
-        + "marriedFemaleEmps(x, y) as (select * from femaleEmps where maritaStatus = 'M')\n"
-        + "select deptno from femaleEmps",
+            + "marriedFemaleEmps(x, y) as (select * from femaleEmps where maritaStatus = 'M')\n"
+            + "select deptno from femaleEmps",
         "WITH `FEMALEEMPS` AS (SELECT *\n"
-        + "FROM `EMPS`\n"
-        + "WHERE (`GENDER` = 'F')), `MARRIEDFEMALEEMPS` (`X`, `Y`) AS (SELECT *\n"
-        + "FROM `FEMALEEMPS`\n"
-        + "WHERE (`MARITASTATUS` = 'M')) SELECT `DEPTNO`\n"
-        + "FROM `FEMALEEMPS`");
+            + "FROM `EMPS`\n"
+            + "WHERE (`GENDER` = 'F')), `MARRIEDFEMALEEMPS` (`X`, `Y`) AS (SELECT *\n"
+            + "FROM `FEMALEEMPS`\n"
+            + "WHERE (`MARITASTATUS` = 'M')) SELECT `DEPTNO`\n"
+            + "FROM `FEMALEEMPS`");
   }
 
   @Test public void testWithFails() {
     checkFails("with femaleEmps as ^select^ * from emps where gender = 'F'\n"
-        + "select deptno from femaleEmps",
+            + "select deptno from femaleEmps",
         "(?s)Encountered \"select\" at .*");
   }
 
   @Test public void testWithValues() {
     check(
         "with v(i,c) as (values (1, 'a'), (2, 'bb'))\n"
-        + "select c, i from v",
+            + "select c, i from v",
         "WITH `V` (`I`, `C`) AS (VALUES (ROW(1, 'a')), (ROW(2, 'bb'))) SELECT `C`, `I`\n"
-        + "FROM `V`");
+            + "FROM `V`");
   }
 
   @Test public void testWithNestedFails() {
     // SQL standard does not allow WITH to contain WITH
     checkFails("with emp2 as (select * from emp)\n"
-        + "^with^ dept2 as (select * from dept)\n"
-        + "select 1 as one from emp, dept",
+            + "^with^ dept2 as (select * from dept)\n"
+            + "select 1 as one from emp, dept",
         "(?s)Encountered \"with\" at .*");
   }
 
   @Test public void testWithNestedInSubquery() {
     // SQL standard does not allow sub-query to contain WITH but we do
     check("with emp2 as (select * from emp)\n"
-        + "(\n"
-        + "  with dept2 as (select * from dept)\n"
-        + "  select 1 as one from empDept)",
+            + "(\n"
+            + "  with dept2 as (select * from dept)\n"
+            + "  select 1 as one from empDept)",
         "WITH `EMP2` AS (SELECT *\n"
-        + "FROM `EMP`) WITH `DEPT2` AS (SELECT *\n"
-        + "FROM `DEPT`) SELECT 1 AS `ONE`\n"
-        + "FROM `EMPDEPT`");
+            + "FROM `EMP`) WITH `DEPT2` AS (SELECT *\n"
+            + "FROM `DEPT`) SELECT 1 AS `ONE`\n"
+            + "FROM `EMPDEPT`");
   }
 
   @Test public void testWithUnion() {
     // Per the standard WITH ... SELECT ... UNION is valid even without parens.
     check("with emp2 as (select * from emp)\n"
-        + "select * from emp2\n"
-        + "union\n"
-        + "select * from emp2\n",
+            + "select * from emp2\n"
+            + "union\n"
+            + "select * from emp2\n",
         "WITH `EMP2` AS (SELECT *\n"
-        + "FROM `EMP`) (SELECT *\n"
-        + "FROM `EMP2`\n"
-        + "UNION\n"
-        + "SELECT *\n"
-        + "FROM `EMP2`)");
+            + "FROM `EMP`) (SELECT *\n"
+            + "FROM `EMP2`\n"
+            + "UNION\n"
+            + "SELECT *\n"
+            + "FROM `EMP2`)");
   }
 
   @Test public void testIdentifier() {
@@ -877,12 +883,12 @@ public class SqlParserTest {
     // is a table alias.
     check("select * from myMap[field], myArray[1 + 2]",
         "SELECT *\n"
-        + "FROM `MYMAP` AS `field`,\n"
-        + "`MYARRAY` AS `1 + 2`");
+            + "FROM `MYMAP` AS `field`,\n"
+            + "`MYARRAY` AS `1 + 2`");
     check("select * from myMap [field], myArray [1 + 2]",
         "SELECT *\n"
-        + "FROM `MYMAP` AS `field`,\n"
-        + "`MYARRAY` AS `1 + 2`");
+            + "FROM `MYMAP` AS `field`,\n"
+            + "`MYARRAY` AS `1 + 2`");
   }
 
   @Test public void testBackTickQuery() {
@@ -890,16 +896,16 @@ public class SqlParserTest {
     check(
         "select `x`.`b baz` from `emp` as `x` where `x`.deptno in (10, 20)",
         "SELECT `x`.`b baz`\n"
-        + "FROM `emp` AS `x`\n"
-        + "WHERE (`x`.`DEPTNO` IN (10, 20))");
+            + "FROM `emp` AS `x`\n"
+            + "WHERE (`x`.`DEPTNO` IN (10, 20))");
   }
 
   @Test public void testInList() {
     check(
         "select * from emp where deptno in (10, 20) and gender = 'F'",
         "SELECT *\n"
-        + "FROM `EMP`\n"
-        + "WHERE ((`DEPTNO` IN (10, 20)) AND (`GENDER` = 'F'))");
+            + "FROM `EMP`\n"
+            + "WHERE ((`DEPTNO` IN (10, 20)) AND (`GENDER` = 'F'))");
   }
 
   @Test public void testInListEmptyFails() {
@@ -912,9 +918,9 @@ public class SqlParserTest {
     check(
         "select * from emp where deptno in (select deptno from dept)",
         "SELECT *\n"
-        + "FROM `EMP`\n"
-        + "WHERE (`DEPTNO` IN (SELECT `DEPTNO`\n"
-        + "FROM `DEPT`))");
+            + "FROM `EMP`\n"
+            + "WHERE (`DEPTNO` IN (SELECT `DEPTNO`\n"
+            + "FROM `DEPT`))");
   }
 
   /**
@@ -924,64 +930,64 @@ public class SqlParserTest {
     check(
         "select * from emp where deptno in (select deptno from dept group by 1, 2)",
         "SELECT *\n"
-        + "FROM `EMP`\n"
-        + "WHERE (`DEPTNO` IN (SELECT `DEPTNO`\n"
-        + "FROM `DEPT`\n"
-        + "GROUP BY 1, 2))");
+            + "FROM `EMP`\n"
+            + "WHERE (`DEPTNO` IN (SELECT `DEPTNO`\n"
+            + "FROM `DEPT`\n"
+            + "GROUP BY 1, 2))");
   }
 
   @Test public void testInSetop() {
     check(
         "select * from emp where deptno in ((select deptno from dept union select * from dept)"
-        + "except select * from dept) and false",
+            + "except select * from dept) and false",
         "SELECT *\n"
-        + "FROM `EMP`\n"
-        + "WHERE ((`DEPTNO` IN ((SELECT `DEPTNO`\n"
-        + "FROM `DEPT`\n"
-        + "UNION\n"
-        + "SELECT *\n"
-        + "FROM `DEPT`)\n"
-        + "EXCEPT\n"
-        + "SELECT *\n"
-        + "FROM `DEPT`)) AND FALSE)");
+            + "FROM `EMP`\n"
+            + "WHERE ((`DEPTNO` IN ((SELECT `DEPTNO`\n"
+            + "FROM `DEPT`\n"
+            + "UNION\n"
+            + "SELECT *\n"
+            + "FROM `DEPT`)\n"
+            + "EXCEPT\n"
+            + "SELECT *\n"
+            + "FROM `DEPT`)) AND FALSE)");
   }
 
   @Test public void testUnion() {
     check(
         "select * from a union select * from a",
         "(SELECT *\n"
-        + "FROM `A`\n"
-        + "UNION\n"
-        + "SELECT *\n"
-        + "FROM `A`)");
+            + "FROM `A`\n"
+            + "UNION\n"
+            + "SELECT *\n"
+            + "FROM `A`)");
     check(
         "select * from a union all select * from a",
         "(SELECT *\n"
-        + "FROM `A`\n"
-        + "UNION ALL\n"
-        + "SELECT *\n"
-        + "FROM `A`)");
+            + "FROM `A`\n"
+            + "UNION ALL\n"
+            + "SELECT *\n"
+            + "FROM `A`)");
     check(
         "select * from a union distinct select * from a",
         "(SELECT *\n"
-        + "FROM `A`\n"
-        + "UNION\n"
-        + "SELECT *\n"
-        + "FROM `A`)");
+            + "FROM `A`\n"
+            + "UNION\n"
+            + "SELECT *\n"
+            + "FROM `A`)");
   }
 
   @Test public void testUnionOrder() {
     check(
         "select a, b from t "
-        + "union all "
-        + "select x, y from u "
-        + "order by 1 asc, 2 desc",
+            + "union all "
+            + "select x, y from u "
+            + "order by 1 asc, 2 desc",
         "(SELECT `A`, `B`\n"
-        + "FROM `T`\n"
-        + "UNION ALL\n"
-        + "SELECT `X`, `Y`\n"
-        + "FROM `U`)\n"
-        + "ORDER BY 1, 2 DESC");
+            + "FROM `T`\n"
+            + "UNION ALL\n"
+            + "SELECT `X`, `Y`\n"
+            + "FROM `U`)\n"
+            + "ORDER BY 1, 2 DESC");
   }
 
   @Test public void testUnionOfNonQueryFails() {
@@ -1007,65 +1013,65 @@ public class SqlParserTest {
     check(
         "select * from a except select * from a",
         "(SELECT *\n"
-        + "FROM `A`\n"
-        + "EXCEPT\n"
-        + "SELECT *\n"
-        + "FROM `A`)");
+            + "FROM `A`\n"
+            + "EXCEPT\n"
+            + "SELECT *\n"
+            + "FROM `A`)");
     check(
         "select * from a except all select * from a",
         "(SELECT *\n"
-        + "FROM `A`\n"
-        + "EXCEPT ALL\n"
-        + "SELECT *\n"
-        + "FROM `A`)");
+            + "FROM `A`\n"
+            + "EXCEPT ALL\n"
+            + "SELECT *\n"
+            + "FROM `A`)");
     check(
         "select * from a except distinct select * from a",
         "(SELECT *\n"
-        + "FROM `A`\n"
-        + "EXCEPT\n"
-        + "SELECT *\n"
-        + "FROM `A`)");
+            + "FROM `A`\n"
+            + "EXCEPT\n"
+            + "SELECT *\n"
+            + "FROM `A`)");
   }
 
   @Test public void testIntersect() {
     check(
         "select * from a intersect select * from a",
         "(SELECT *\n"
-        + "FROM `A`\n"
-        + "INTERSECT\n"
-        + "SELECT *\n"
-        + "FROM `A`)");
+            + "FROM `A`\n"
+            + "INTERSECT\n"
+            + "SELECT *\n"
+            + "FROM `A`)");
     check(
         "select * from a intersect all select * from a",
         "(SELECT *\n"
-        + "FROM `A`\n"
-        + "INTERSECT ALL\n"
-        + "SELECT *\n"
-        + "FROM `A`)");
+            + "FROM `A`\n"
+            + "INTERSECT ALL\n"
+            + "SELECT *\n"
+            + "FROM `A`)");
     check(
         "select * from a intersect distinct select * from a",
         "(SELECT *\n"
-        + "FROM `A`\n"
-        + "INTERSECT\n"
-        + "SELECT *\n"
-        + "FROM `A`)");
+            + "FROM `A`\n"
+            + "INTERSECT\n"
+            + "SELECT *\n"
+            + "FROM `A`)");
   }
 
   @Test public void testJoinCross() {
     check(
         "select * from a as a2 cross join b",
         "SELECT *\n"
-        + "FROM `A` AS `A2`\n"
-        + "CROSS JOIN `B`");
+            + "FROM `A` AS `A2`\n"
+            + "CROSS JOIN `B`");
   }
 
   @Test public void testJoinOn() {
     check(
         "select * from a left join b on 1 = 1 and 2 = 2 where 3 = 3",
         "SELECT *\n"
-        + "FROM `A`\n"
-        + "LEFT JOIN `B` ON ((1 = 1) AND (2 = 2))\n"
-        + "WHERE (3 = 3)");
+            + "FROM `A`\n"
+            + "LEFT JOIN `B` ON ((1 = 1) AND (2 = 2))\n"
+            + "WHERE (3 = 3)");
   }
 
   @Test public void testJoinOnParentheses() {
@@ -1074,12 +1080,12 @@ public class SqlParserTest {
     }
     check(
         "select * from a\n"
-        + " left join (b join c as c1 on 1 = 1) on 2 = 2\n"
-        + "where 3 = 3",
+            + " left join (b join c as c1 on 1 = 1) on 2 = 2\n"
+            + "where 3 = 3",
         "SELECT *\n"
-        + "FROM `A`\n"
-        + "LEFT JOIN (`B` INNER JOIN `C` AS `C1` ON (1 = 1)) ON (2 = 2)\n"
-        + "WHERE (3 = 3)");
+            + "FROM `A`\n"
+            + "LEFT JOIN (`B` INNER JOIN `C` AS `C1` ON (1 = 1)) ON (2 = 2)\n"
+            + "WHERE (3 = 3)");
   }
 
   /**
@@ -1091,22 +1097,22 @@ public class SqlParserTest {
     }
     check(
         "select * from a\n"
-        + " left join (b as b1 (x, y) join (select * from c) c1 on 1 = 1) on 2 = 2\n"
-        + "where 3 = 3",
+            + " left join (b as b1 (x, y) join (select * from c) c1 on 1 = 1) on 2 = 2\n"
+            + "where 3 = 3",
         "SELECT *\n"
-        + "FROM `A`\n"
-        + "LEFT JOIN (`B` AS `B1` (`X`, `Y`) INNER JOIN (SELECT *\n"
-        + "FROM `C`) AS `C1` ON (1 = 1)) ON (2 = 2)\n"
-        + "WHERE (3 = 3)");
+            + "FROM `A`\n"
+            + "LEFT JOIN (`B` AS `B1` (`X`, `Y`) INNER JOIN (SELECT *\n"
+            + "FROM `C`) AS `C1` ON (1 = 1)) ON (2 = 2)\n"
+            + "WHERE (3 = 3)");
   }
 
   @Test public void testExplicitTableInJoin() {
     check(
         "select * from a left join (table b) on 2 = 2 where 3 = 3",
         "SELECT *\n"
-        + "FROM `A`\n"
-        + "LEFT JOIN (TABLE `B`) ON (2 = 2)\n"
-        + "WHERE (3 = 3)");
+            + "FROM `A`\n"
+            + "LEFT JOIN (TABLE `B`) ON (2 = 2)\n"
+            + "WHERE (3 = 3)");
   }
 
   @Test public void testSubqueryInJoin() {
@@ -1115,32 +1121,32 @@ public class SqlParserTest {
     }
     check(
         "select * from (select * from a cross join b) as ab\n"
-        + " left join ((table c) join d on 2 = 2) on 3 = 3\n"
-        + " where 4 = 4",
+            + " left join ((table c) join d on 2 = 2) on 3 = 3\n"
+            + " where 4 = 4",
         "SELECT *\n"
-        + "FROM (SELECT *\n"
-        + "FROM `A`\n"
-        + "CROSS JOIN `B`) AS `AB`\n"
-        + "LEFT JOIN ((TABLE `C`) INNER JOIN `D` ON (2 = 2)) ON (3 = 3)\n"
-        + "WHERE (4 = 4)");
+            + "FROM (SELECT *\n"
+            + "FROM `A`\n"
+            + "CROSS JOIN `B`) AS `AB`\n"
+            + "LEFT JOIN ((TABLE `C`) INNER JOIN `D` ON (2 = 2)) ON (3 = 3)\n"
+            + "WHERE (4 = 4)");
   }
 
   @Test public void testOuterJoinNoiseWord() {
     check(
         "select * from a left outer join b on 1 = 1 and 2 = 2 where 3 = 3",
         "SELECT *\n"
-        + "FROM `A`\n"
-        + "LEFT JOIN `B` ON ((1 = 1) AND (2 = 2))\n"
-        + "WHERE (3 = 3)");
+            + "FROM `A`\n"
+            + "LEFT JOIN `B` ON ((1 = 1) AND (2 = 2))\n"
+            + "WHERE (3 = 3)");
   }
 
   @Test public void testJoinQuery() {
     check(
         "select * from a join (select * from b) as b2 on true",
         "SELECT *\n"
-        + "FROM `A`\n"
-        + "INNER JOIN (SELECT *\n"
-        + "FROM `B`) AS `B2` ON TRUE");
+            + "FROM `A`\n"
+            + "INNER JOIN (SELECT *\n"
+            + "FROM `B`) AS `B2` ON TRUE");
   }
 
   @Test public void testFullInnerJoinFails() {
@@ -1155,8 +1161,8 @@ public class SqlParserTest {
     check(
         "select * from a full outer join b",
         "SELECT *\n"
-        + "FROM `A`\n"
-        + "FULL JOIN `B`");
+            + "FROM `A`\n"
+            + "FULL JOIN `B`");
   }
 
   @Test public void testInnerOuterJoinFails() {
@@ -1172,19 +1178,19 @@ public class SqlParserTest {
     check(
         "select * from (a natural left join b) left join c on b.c1 = c.c1",
         "SELECT *\n"
-        + "FROM (`A` NATURAL LEFT JOIN `B`) LEFT JOIN `C` ON (`B`.`C1` = `C`.`C1`)\n");
+            + "FROM (`A` NATURAL LEFT JOIN `B`) LEFT JOIN `C` ON (`B`.`C1` = `C`.`C1`)\n");
 
     // 2. parens needed
     check(
         "select * from a natural left join (b left join c on b.c1 = c.c1)",
         "SELECT *\n"
-        + "FROM (`A` NATURAL LEFT JOIN `B`) LEFT JOIN `C` ON (`B`.`C1` = `C`.`C1`)\n");
+            + "FROM (`A` NATURAL LEFT JOIN `B`) LEFT JOIN `C` ON (`B`.`C1` = `C`.`C1`)\n");
 
     // 3. same as 1
     check(
         "select * from a natural left join b left join c on b.c1 = c.c1",
         "SELECT *\n"
-        + "FROM (`A` NATURAL LEFT JOIN `B`) LEFT JOIN `C` ON (`B`.`C1` = `C`.`C1`)\n");
+            + "FROM (`A` NATURAL LEFT JOIN `B`) LEFT JOIN `C` ON (`B`.`C1` = `C`.`C1`)\n");
   }
 
   // Note: "select * from a natural cross join b" is actually illegal SQL
@@ -1194,16 +1200,16 @@ public class SqlParserTest {
     check(
         "select * from a natural cross join b",
         "SELECT *\n"
-        + "FROM `A`\n"
-        + "NATURAL CROSS JOIN `B`");
+            + "FROM `A`\n"
+            + "NATURAL CROSS JOIN `B`");
   }
 
   @Test public void testJoinUsing() {
     check(
         "select * from a join b using (x)",
         "SELECT *\n"
-        + "FROM `A`\n"
-        + "INNER JOIN `B` USING (`X`)");
+            + "FROM `A`\n"
+            + "INNER JOIN `B` USING (`X`)");
     checkFails(
         "select * from a join b using (^)^ where c = d",
         "(?s).*Encountered \"[)]\" at line 1, column 31.*");
@@ -1212,31 +1218,31 @@ public class SqlParserTest {
   @Test public void testTableSample() {
     check(
         "select * from ("
-        + "  select * "
-        + "  from emp "
-        + "  join dept on emp.deptno = dept.deptno"
-        + "  where gender = 'F'"
-        + "  order by sal) tablesample substitute('medium')",
+            + "  select * "
+            + "  from emp "
+            + "  join dept on emp.deptno = dept.deptno"
+            + "  where gender = 'F'"
+            + "  order by sal) tablesample substitute('medium')",
         "SELECT *\n"
-        + "FROM (SELECT *\n"
-        + "FROM `EMP`\n"
-        + "INNER JOIN `DEPT` ON (`EMP`.`DEPTNO` = `DEPT`.`DEPTNO`)\n"
-        + "WHERE (`GENDER` = 'F')\n"
-        + "ORDER BY `SAL`) TABLESAMPLE SUBSTITUTE('MEDIUM')");
+            + "FROM (SELECT *\n"
+            + "FROM `EMP`\n"
+            + "INNER JOIN `DEPT` ON (`EMP`.`DEPTNO` = `DEPT`.`DEPTNO`)\n"
+            + "WHERE (`GENDER` = 'F')\n"
+            + "ORDER BY `SAL`) TABLESAMPLE SUBSTITUTE('MEDIUM')");
 
     check(
         "select * "
-        + "from emp as x tablesample substitute('medium') "
-        + "join dept tablesample substitute('lar' /* split */ 'ge') on x.deptno = dept.deptno",
+            + "from emp as x tablesample substitute('medium') "
+            + "join dept tablesample substitute('lar' /* split */ 'ge') on x.deptno = dept.deptno",
         "SELECT *\n"
-        + "FROM `EMP` AS `X` TABLESAMPLE SUBSTITUTE('MEDIUM')\n"
-        + "INNER JOIN `DEPT` TABLESAMPLE SUBSTITUTE('LARGE') ON (`X`.`DEPTNO` = `DEPT`.`DEPTNO`)");
+            + "FROM `EMP` AS `X` TABLESAMPLE SUBSTITUTE('MEDIUM')\n"
+            + "INNER JOIN `DEPT` TABLESAMPLE SUBSTITUTE('LARGE') ON (`X`.`DEPTNO` = `DEPT`.`DEPTNO`)");
 
     check(
         "select * "
-        + "from emp as x tablesample bernoulli(50)",
+            + "from emp as x tablesample bernoulli(50)",
         "SELECT *\n"
-        + "FROM `EMP` AS `X` TABLESAMPLE BERNOULLI(50.0)");
+            + "FROM `EMP` AS `X` TABLESAMPLE BERNOULLI(50.0)");
   }
 
   @Test public void testLiteral() {
@@ -1245,7 +1251,7 @@ public class SqlParserTest {
     check(
         "select 1 as one, 'x' as x, null as n from emp",
         "SELECT 1 AS `ONE`, 'x' AS `X`, NULL AS `N`\n"
-        + "FROM `EMP`");
+            + "FROM `EMP`");
 
     // Even though it looks like a date, it's just a string.
     checkExp("'2004-06-01'", "'2004-06-01'");
@@ -1293,72 +1299,72 @@ public class SqlParserTest {
     check(
         "select * from a join b using (x), c join d using (y)",
         "SELECT *\n"
-        + "FROM `A`\n"
-        + "INNER JOIN `B` USING (`X`),\n"
-        + "`C`\n"
-        + "INNER JOIN `D` USING (`Y`)");
+            + "FROM `A`\n"
+            + "INNER JOIN `B` USING (`X`),\n"
+            + "`C`\n"
+            + "INNER JOIN `D` USING (`Y`)");
   }
 
   @Test public void testMixedStar() {
     check(
         "select emp.*, 1 as foo from emp, dept",
         "SELECT `EMP`.*, 1 AS `FOO`\n"
-        + "FROM `EMP`,\n"
-        + "`DEPT`");
+            + "FROM `EMP`,\n"
+            + "`DEPT`");
   }
 
   @Test public void testNotExists() {
     check(
         "select * from dept where not not exists (select * from emp) and true",
         "SELECT *\n"
-        + "FROM `DEPT`\n"
-        + "WHERE ((NOT (NOT (EXISTS (SELECT *\n"
-        + "FROM `EMP`)))) AND TRUE)");
+            + "FROM `DEPT`\n"
+            + "WHERE ((NOT (NOT (EXISTS (SELECT *\n"
+            + "FROM `EMP`)))) AND TRUE)");
   }
 
   @Test public void testOrder() {
     check(
         "select * from emp order by empno, gender desc, deptno asc, empno asc, name desc",
         "SELECT *\n"
-        + "FROM `EMP`\n"
-        + "ORDER BY `EMPNO`, `GENDER` DESC, `DEPTNO`, `EMPNO`, `NAME` DESC");
+            + "FROM `EMP`\n"
+            + "ORDER BY `EMPNO`, `GENDER` DESC, `DEPTNO`, `EMPNO`, `NAME` DESC");
   }
 
   @Test public void testOrderNullsFirst() {
     check(
         "select * from emp order by gender desc nulls last, deptno asc nulls first, empno nulls last",
         "SELECT *\n"
-        + "FROM `EMP`\n"
-        + "ORDER BY `GENDER` DESC NULLS LAST, `DEPTNO` NULLS FIRST, `EMPNO` NULLS LAST");
+            + "FROM `EMP`\n"
+            + "ORDER BY `GENDER` DESC NULLS LAST, `DEPTNO` NULLS FIRST, `EMPNO` NULLS LAST");
   }
 
   @Test public void testOrderInternal() {
     check(
         "(select * from emp order by empno) union select * from emp",
         "((SELECT *\n"
-        + "FROM `EMP`\n"
-        + "ORDER BY `EMPNO`)\n"
-        + "UNION\n"
-        + "SELECT *\n"
-        + "FROM `EMP`)");
+            + "FROM `EMP`\n"
+            + "ORDER BY `EMPNO`)\n"
+            + "UNION\n"
+            + "SELECT *\n"
+            + "FROM `EMP`)");
 
     check(
         "select * from (select * from t order by x, y) where a = b",
         "SELECT *\n"
-        + "FROM (SELECT *\n"
-        + "FROM `T`\n"
-        + "ORDER BY `X`, `Y`)\n"
-        + "WHERE (`A` = `B`)");
+            + "FROM (SELECT *\n"
+            + "FROM `T`\n"
+            + "ORDER BY `X`, `Y`)\n"
+            + "WHERE (`A` = `B`)");
   }
 
   @Test public void testOrderIllegalInExpression() {
     check(
         "select (select 1 from foo order by x,y) from t where a = b",
         "SELECT (SELECT 1\n"
-        + "FROM `FOO`\n"
-        + "ORDER BY `X`, `Y`)\n"
-        + "FROM `T`\n"
-        + "WHERE (`A` = `B`)");
+            + "FROM `FOO`\n"
+            + "ORDER BY `X`, `Y`)\n"
+            + "FROM `T`\n"
+            + "WHERE (`A` = `B`)");
     checkFails(
         "select (1 ^order^ by x, y) from t where a = b",
         "ORDER BY unexpected");
@@ -1368,62 +1374,62 @@ public class SqlParserTest {
     check(
         "select a from foo order by b, c offset 1 row fetch first 2 row only",
         "SELECT `A`\n"
-        + "FROM `FOO`\n"
-        + "ORDER BY `B`, `C`\n"
-        + "OFFSET 1 ROWS\n"
-        + "FETCH NEXT 2 ROWS ONLY");
+            + "FROM `FOO`\n"
+            + "ORDER BY `B`, `C`\n"
+            + "OFFSET 1 ROWS\n"
+            + "FETCH NEXT 2 ROWS ONLY");
     // as above, but ROWS rather than ROW
     check(
         "select a from foo order by b, c offset 1 rows fetch first 2 rows only",
         "SELECT `A`\n"
-        + "FROM `FOO`\n"
-        + "ORDER BY `B`, `C`\n"
-        + "OFFSET 1 ROWS\n"
-        + "FETCH NEXT 2 ROWS ONLY");
+            + "FROM `FOO`\n"
+            + "ORDER BY `B`, `C`\n"
+            + "OFFSET 1 ROWS\n"
+            + "FETCH NEXT 2 ROWS ONLY");
     // as above, but NEXT (means same as FIRST)
     check(
         "select a from foo order by b, c offset 1 rows fetch next 3 rows only",
         "SELECT `A`\n"
-        + "FROM `FOO`\n"
-        + "ORDER BY `B`, `C`\n"
-        + "OFFSET 1 ROWS\n"
-        + "FETCH NEXT 3 ROWS ONLY");
+            + "FROM `FOO`\n"
+            + "ORDER BY `B`, `C`\n"
+            + "OFFSET 1 ROWS\n"
+            + "FETCH NEXT 3 ROWS ONLY");
     // as above, but omit the ROWS noise word after OFFSET. This is not
     // compatible with SQL:2008 but allows the Postgres syntax
     // "LIMIT ... OFFSET".
     check(
         "select a from foo order by b, c offset 1 fetch next 3 rows only",
         "SELECT `A`\n"
-        + "FROM `FOO`\n"
-        + "ORDER BY `B`, `C`\n"
-        + "OFFSET 1 ROWS\n"
-        + "FETCH NEXT 3 ROWS ONLY");
+            + "FROM `FOO`\n"
+            + "ORDER BY `B`, `C`\n"
+            + "OFFSET 1 ROWS\n"
+            + "FETCH NEXT 3 ROWS ONLY");
     // as above, omit OFFSET
     check(
         "select a from foo order by b, c fetch next 3 rows only",
         "SELECT `A`\n"
-        + "FROM `FOO`\n"
-        + "ORDER BY `B`, `C`\n"
-        + "FETCH NEXT 3 ROWS ONLY");
+            + "FROM `FOO`\n"
+            + "ORDER BY `B`, `C`\n"
+            + "FETCH NEXT 3 ROWS ONLY");
     // FETCH, no ORDER BY or OFFSET
     check(
         "select a from foo fetch next 4 rows only",
         "SELECT `A`\n"
-        + "FROM `FOO`\n"
-        + "FETCH NEXT 4 ROWS ONLY");
+            + "FROM `FOO`\n"
+            + "FETCH NEXT 4 ROWS ONLY");
     // OFFSET, no ORDER BY or FETCH
     check(
         "select a from foo offset 1 row",
         "SELECT `A`\n"
-        + "FROM `FOO`\n"
-        + "OFFSET 1 ROWS");
+            + "FROM `FOO`\n"
+            + "OFFSET 1 ROWS");
     // OFFSET and FETCH, no ORDER BY
     check(
         "select a from foo offset 1 row fetch next 3 rows only",
         "SELECT `A`\n"
-        + "FROM `FOO`\n"
-        + "OFFSET 1 ROWS\n"
-        + "FETCH NEXT 3 ROWS ONLY");
+            + "FROM `FOO`\n"
+            + "OFFSET 1 ROWS\n"
+            + "FETCH NEXT 3 ROWS ONLY");
     // missing ROWS after FETCH
     checkFails(
         "select a from foo offset 1 fetch next 3 ^only^",
@@ -1443,39 +1449,39 @@ public class SqlParserTest {
     check(
         "select a from foo order by b, c limit 2 offset 1",
         "SELECT `A`\n"
-        + "FROM `FOO`\n"
-        + "ORDER BY `B`, `C`\n"
-        + "OFFSET 1 ROWS\n"
-        + "FETCH NEXT 2 ROWS ONLY");
+            + "FROM `FOO`\n"
+            + "ORDER BY `B`, `C`\n"
+            + "OFFSET 1 ROWS\n"
+            + "FETCH NEXT 2 ROWS ONLY");
     check(
         "select a from foo order by b, c limit 2",
         "SELECT `A`\n"
-        + "FROM `FOO`\n"
-        + "ORDER BY `B`, `C`\n"
-        + "FETCH NEXT 2 ROWS ONLY");
+            + "FROM `FOO`\n"
+            + "ORDER BY `B`, `C`\n"
+            + "FETCH NEXT 2 ROWS ONLY");
     check(
         "select a from foo order by b, c offset 1",
         "SELECT `A`\n"
-        + "FROM `FOO`\n"
-        + "ORDER BY `B`, `C`\n"
-        + "OFFSET 1 ROWS");
+            + "FROM `FOO`\n"
+            + "ORDER BY `B`, `C`\n"
+            + "OFFSET 1 ROWS");
   }
 
   @Test public void testSqlInlineComment() {
     check(
         "select 1 from t --this is a comment\n",
         "SELECT 1\n"
-        + "FROM `T`");
+            + "FROM `T`");
     check(
         "select 1 from t--\n",
         "SELECT 1\n"
-        + "FROM `T`");
+            + "FROM `T`");
     check(
         "select 1 from t--this is a comment\n"
-        + "where a>b-- this is comment\n",
+            + "where a>b-- this is comment\n",
         "SELECT 1\n"
-        + "FROM `T`\n"
-        + "WHERE (`A` > `B`)");
+            + "FROM `T`\n"
+            + "WHERE (`A` > `B`)");
   }
 
   @Test public void testMultilineComment() {
@@ -1483,15 +1489,15 @@ public class SqlParserTest {
     check(
         "select 1 /* , 2 */, 3 from t",
         "SELECT 1, 3\n"
-        + "FROM `T`");
+            + "FROM `T`");
 
     // on several lines
     check(
         "select /* 1,\n"
-        + " 2, \n"
-        + " */ 3 from t",
+            + " 2, \n"
+            + " */ 3 from t",
         "SELECT 3\n"
-        + "FROM `T`");
+            + "FROM `T`");
 
     // stuff inside comment
     check(
@@ -1508,12 +1514,12 @@ public class SqlParserTest {
 
     check(
         "values (- -1\n"
-        + ")",
+            + ")",
         "(VALUES (ROW((- -1))))");
 
     check(
         "values (--1+\n"
-        + "2)",
+            + "2)",
         "(VALUES (ROW(2)))");
 
     // end of multiline commment without start
@@ -1544,7 +1550,7 @@ public class SqlParserTest {
     if (Bug.FRG73_FIXED) {
       checkFails(
           "values /* multiline contains -- singline */ \n"
-          + " (1)",
+              + " (1)",
           "xxx");
     }
 
@@ -1554,25 +1560,25 @@ public class SqlParserTest {
       // erroneous token.
       checkFails(
           "values ( -- rest of line /* a comment  \n"
-          + " 1, ^*/^ 2)",
+              + " 1, ^*/^ 2)",
           "Encountered \"/\\*\" at");
     }
 
     check(
         "values (1 + /* comment -- rest of line\n"
-        + " rest of comment */ 2)",
+            + " rest of comment */ 2)",
         "(VALUES (ROW((1 + 2))))");
 
     // multiline comment inside singleline comment
     check(
         "values -- rest of line /* a comment */ \n"
-        + "(1)",
+            + "(1)",
         "(VALUES (ROW(1)))");
 
     // non-terminated multiline comment inside singleline comment
     check(
         "values -- rest of line /* a comment  \n"
-        + "(1)",
+            + "(1)",
         "(VALUES (ROW(1)))");
 
     // even if comment abuts the tokens at either end, it becomes a space
@@ -1624,7 +1630,7 @@ public class SqlParserTest {
         "((((1 + 2.3E-4) + 5E-7) + 0.7) + 8)");
     checkExp(
         "1- -2.3e-4 - -.5e-6  -\n"
-        + "-.7++8",
+            + "-.7++8",
         "((((1 - -2.3E-4) - -5E-7) - -0.7) + 8)");
     checkExp("1+-2.*-3.e-1/-4", "(1 + ((-2 * -3E-1) / -4))");
   }
@@ -1668,32 +1674,32 @@ public class SqlParserTest {
   @Test public void testPrecedenceSetOps() {
     check(
         "select * from a union "
-        + "select * from b intersect "
-        + "select * from c intersect "
-        + "select * from d except "
-        + "select * from e except "
-        + "select * from f union "
-        + "select * from g",
+            + "select * from b intersect "
+            + "select * from c intersect "
+            + "select * from d except "
+            + "select * from e except "
+            + "select * from f union "
+            + "select * from g",
         "((((SELECT *\n"
-        + "FROM `A`\n"
-        + "UNION\n"
-        + "((SELECT *\n"
-        + "FROM `B`\n"
-        + "INTERSECT\n"
-        + "SELECT *\n"
-        + "FROM `C`)\n"
-        + "INTERSECT\n"
-        + "SELECT *\n"
-        + "FROM `D`))\n"
-        + "EXCEPT\n"
-        + "SELECT *\n"
-        + "FROM `E`)\n"
-        + "EXCEPT\n"
-        + "SELECT *\n"
-        + "FROM `F`)\n"
-        + "UNION\n"
-        + "SELECT *\n"
-        + "FROM `G`)");
+            + "FROM `A`\n"
+            + "UNION\n"
+            + "((SELECT *\n"
+            + "FROM `B`\n"
+            + "INTERSECT\n"
+            + "SELECT *\n"
+            + "FROM `C`)\n"
+            + "INTERSECT\n"
+            + "SELECT *\n"
+            + "FROM `D`))\n"
+            + "EXCEPT\n"
+            + "SELECT *\n"
+            + "FROM `E`)\n"
+            + "EXCEPT\n"
+            + "SELECT *\n"
+            + "FROM `F`)\n"
+            + "UNION\n"
+            + "SELECT *\n"
+            + "FROM `G`)");
   }
 
   @Test public void testQueryInFrom() {
@@ -1701,10 +1707,10 @@ public class SqlParserTest {
     check(
         "select * from (select * from emp) as e join (select * from dept) d",
         "SELECT *\n"
-        + "FROM (SELECT *\n"
-        + "FROM `EMP`) AS `E`\n"
-        + "INNER JOIN (SELECT *\n"
-        + "FROM `DEPT`) AS `D`");
+            + "FROM (SELECT *\n"
+            + "FROM `EMP`) AS `E`\n"
+            + "INNER JOIN (SELECT *\n"
+            + "FROM `DEPT`) AS `D`");
   }
 
   @Test public void testQuotesInString() {
@@ -1720,34 +1726,34 @@ public class SqlParserTest {
     check(
         "select * from emp where 3 = (select count(*) from dept where dept.deptno = emp.deptno)",
         "SELECT *\n"
-        + "FROM `EMP`\n"
-        + "WHERE (3 = (SELECT COUNT(*)\n"
-        + "FROM `DEPT`\n"
-        + "WHERE (`DEPT`.`DEPTNO` = `EMP`.`DEPTNO`)))");
+            + "FROM `EMP`\n"
+            + "WHERE (3 = (SELECT COUNT(*)\n"
+            + "FROM `DEPT`\n"
+            + "WHERE (`DEPT`.`DEPTNO` = `EMP`.`DEPTNO`)))");
   }
 
   @Test public void testScalarQueryInSelect() {
     check(
         "select x, (select count(*) from dept where dept.deptno = emp.deptno) from emp",
         "SELECT `X`, (SELECT COUNT(*)\n"
-        + "FROM `DEPT`\n"
-        + "WHERE (`DEPT`.`DEPTNO` = `EMP`.`DEPTNO`))\n"
-        + "FROM `EMP`");
+            + "FROM `DEPT`\n"
+            + "WHERE (`DEPT`.`DEPTNO` = `EMP`.`DEPTNO`))\n"
+            + "FROM `EMP`");
   }
 
   @Test public void testSelectList() {
     check(
         "select * from emp, dept",
         "SELECT *\n"
-        + "FROM `EMP`,\n"
-        + "`DEPT`");
+            + "FROM `EMP`,\n"
+            + "`DEPT`");
   }
 
   @Test public void testSelectList3() {
     check(
         "select 1, emp.*, 2 from emp",
         "SELECT 1, `EMP`.*, 2\n"
-        + "FROM `EMP`");
+            + "FROM `EMP`");
   }
 
   @Test public void testSelectList4() {
@@ -1760,14 +1766,14 @@ public class SqlParserTest {
     check(
         "select * from emp",
         "SELECT *\n"
-        + "FROM `EMP`");
+            + "FROM `EMP`");
   }
 
   @Test public void testSelectDistinct() {
     check(
         "select distinct foo from bar",
         "SELECT DISTINCT `FOO`\n"
-        + "FROM `BAR`");
+            + "FROM `BAR`");
   }
 
   @Test public void testSelectAll() {
@@ -1775,24 +1781,24 @@ public class SqlParserTest {
     check(
         "select * from (select all foo from bar) as xyz",
         "SELECT *\n"
-        + "FROM (SELECT ALL `FOO`\n"
-        + "FROM `BAR`) AS `XYZ`");
+            + "FROM (SELECT ALL `FOO`\n"
+            + "FROM `BAR`) AS `XYZ`");
   }
 
   @Test public void testWhere() {
     check(
         "select * from emp where empno > 5 and gender = 'F'",
         "SELECT *\n"
-        + "FROM `EMP`\n"
-        + "WHERE ((`EMPNO` > 5) AND (`GENDER` = 'F'))");
+            + "FROM `EMP`\n"
+            + "WHERE ((`EMPNO` > 5) AND (`GENDER` = 'F'))");
   }
 
   @Test public void testNestedSelect() {
     check(
         "select * from (select * from emp)",
         "SELECT *\n"
-        + "FROM (SELECT *\n"
-        + "FROM `EMP`)");
+            + "FROM (SELECT *\n"
+            + "FROM `EMP`)");
   }
 
   @Test public void testValues() {
@@ -1807,24 +1813,24 @@ public class SqlParserTest {
     check(
         "select * from (values(1,'two'), 3, (4, 'five'))",
         "SELECT *\n"
-        + "FROM (VALUES (ROW(1, 'two')), (ROW(3)), (ROW(4, 'five')))");
+            + "FROM (VALUES (ROW(1, 'two')), (ROW(3)), (ROW(4, 'five')))");
   }
 
   @Test public void testFromValuesWithoutParens() {
     checkFails(
         "select 1 from ^values^('x')",
         "Encountered \"values\" at line 1, column 15\\.\n"
-        + "Was expecting one of:\n"
-        + "    <IDENTIFIER> \\.\\.\\.\n"
-        + "    <QUOTED_IDENTIFIER> \\.\\.\\.\n"
-        + "    <BACK_QUOTED_IDENTIFIER> \\.\\.\\.\n"
-        + "    <BRACKET_QUOTED_IDENTIFIER> \\.\\.\\.\n"
-        + "    <UNICODE_QUOTED_IDENTIFIER> \\.\\.\\.\n"
-        + "    \"LATERAL\" \\.\\.\\.\n"
-        + "    \"\\(\" \\.\\.\\.\n"
-        + "    \"UNNEST\" \\.\\.\\.\n"
-        + "    \"TABLE\" \\.\\.\\.\n"
-        + "    ");
+            + "Was expecting one of:\n"
+            + "    <IDENTIFIER> \\.\\.\\.\n"
+            + "    <QUOTED_IDENTIFIER> \\.\\.\\.\n"
+            + "    <BACK_QUOTED_IDENTIFIER> \\.\\.\\.\n"
+            + "    <BRACKET_QUOTED_IDENTIFIER> \\.\\.\\.\n"
+            + "    <UNICODE_QUOTED_IDENTIFIER> \\.\\.\\.\n"
+            + "    \"LATERAL\" \\.\\.\\.\n"
+            + "    \"\\(\" \\.\\.\\.\n"
+            + "    \"UNNEST\" \\.\\.\\.\n"
+            + "    \"TABLE\" \\.\\.\\.\n"
+            + "    ");
   }
 
   @Test public void testEmptyValues() {
@@ -1846,14 +1852,14 @@ public class SqlParserTest {
     check(
         "table emp order by name",
         "(TABLE `EMP`)\n"
-        + "ORDER BY `NAME`");
+            + "ORDER BY `NAME`");
   }
 
   @Test public void testSelectFromExplicitTable() {
     check(
         "select * from (table emp)",
         "SELECT *\n"
-        + "FROM (TABLE `EMP`)");
+            + "FROM (TABLE `EMP`)");
   }
 
   @Test public void testSelectFromBareExplicitTableFails() {
@@ -1871,24 +1877,24 @@ public class SqlParserTest {
     check(
         "select * from table(ramp(3, 4))",
         "SELECT *\n"
-        + "FROM TABLE(`RAMP`(3, 4))");
+            + "FROM TABLE(`RAMP`(3, 4))");
   }
 
   @Test public void testCollectionTableWithCursorParam() {
     check(
         "select * from table(dedup(cursor(select * from emps),'name'))",
         "SELECT *\n"
-        + "FROM TABLE(`DEDUP`((CURSOR ((SELECT *\n"
-        + "FROM `EMPS`))), 'name'))");
+            + "FROM TABLE(`DEDUP`((CURSOR ((SELECT *\n"
+            + "FROM `EMPS`))), 'name'))");
   }
 
   @Test public void testCollectionTableWithColumnListParam() {
     check(
         "select * from table(dedup(cursor(select * from emps),"
-        + "row(empno, name)))",
+            + "row(empno, name)))",
         "SELECT *\n"
-        + "FROM TABLE(`DEDUP`((CURSOR ((SELECT *\n"
-        + "FROM `EMPS`))), (ROW(`EMPNO`, `NAME`))))");
+            + "FROM TABLE(`DEDUP`((CURSOR ((SELECT *\n"
+            + "FROM `EMPS`))), (ROW(`EMPNO`, `NAME`))))");
   }
 
   @Test public void testIllegalCursors() {
@@ -1907,74 +1913,74 @@ public class SqlParserTest {
     check(
         "explain plan for select * from emps",
         "EXPLAIN PLAN INCLUDING ATTRIBUTES WITH IMPLEMENTATION FOR\n"
-        + "SELECT *\n"
-        + "FROM `EMPS`");
+            + "SELECT *\n"
+            + "FROM `EMPS`");
   }
 
   @Test public void testExplainWithImpl() {
     check(
         "explain plan with implementation for select * from emps",
         "EXPLAIN PLAN INCLUDING ATTRIBUTES WITH IMPLEMENTATION FOR\n"
-        + "SELECT *\n"
-        + "FROM `EMPS`");
+            + "SELECT *\n"
+            + "FROM `EMPS`");
   }
 
   @Test public void testExplainWithoutImpl() {
     check(
         "explain plan without implementation for select * from emps",
         "EXPLAIN PLAN INCLUDING ATTRIBUTES WITHOUT IMPLEMENTATION FOR\n"
-        + "SELECT *\n"
-        + "FROM `EMPS`");
+            + "SELECT *\n"
+            + "FROM `EMPS`");
   }
 
   @Test public void testExplainWithType() {
     check(
         "explain plan with type for (values (true))",
         "EXPLAIN PLAN INCLUDING ATTRIBUTES WITH TYPE FOR\n"
-        + "(VALUES (ROW(TRUE)))");
+            + "(VALUES (ROW(TRUE)))");
   }
 
   @Test public void testInsertSelect() {
     check(
         "insert into emps select * from emps",
         "INSERT INTO `EMPS`\n"
-        + "(SELECT *\n"
-        + "FROM `EMPS`)");
+            + "(SELECT *\n"
+            + "FROM `EMPS`)");
   }
 
   @Test public void testInsertUnion() {
     check(
         "insert into emps select * from emps1 union select * from emps2",
         "INSERT INTO `EMPS`\n"
-        + "(SELECT *\n"
-        + "FROM `EMPS1`\n"
-        + "UNION\n"
-        + "SELECT *\n"
-        + "FROM `EMPS2`)");
+            + "(SELECT *\n"
+            + "FROM `EMPS1`\n"
+            + "UNION\n"
+            + "SELECT *\n"
+            + "FROM `EMPS2`)");
   }
 
   @Test public void testInsertValues() {
     check(
         "insert into emps values (1,'Fredkin')",
         "INSERT INTO `EMPS`\n"
-        + "(VALUES (ROW(1, 'Fredkin')))");
+            + "(VALUES (ROW(1, 'Fredkin')))");
   }
 
   @Test public void testInsertColumnList() {
     check(
         "insert into emps(x,y) select * from emps",
         "INSERT INTO `EMPS` (`X`, `Y`)\n"
-        + "(SELECT *\n"
-        + "FROM `EMPS`)");
+            + "(SELECT *\n"
+            + "FROM `EMPS`)");
   }
 
   @Test public void testExplainInsert() {
     check(
         "explain plan for insert into emps1 select * from emps2",
         "EXPLAIN PLAN INCLUDING ATTRIBUTES WITH IMPLEMENTATION FOR\n"
-        + "INSERT INTO `EMPS1`\n"
-        + "(SELECT *\n"
-        + "FROM `EMPS2`)");
+            + "INSERT INTO `EMPS1`\n"
+            + "(SELECT *\n"
+            + "FROM `EMPS2`)");
   }
 
   @Test public void testDelete() {
@@ -1985,49 +1991,49 @@ public class SqlParserTest {
     check(
         "delete from emps where empno=12",
         "DELETE FROM `EMPS`\n"
-        + "WHERE (`EMPNO` = 12)");
+            + "WHERE (`EMPNO` = 12)");
   }
 
   @Test public void testMergeSelectSource() {
     check(
         "merge into emps e "
-        + "using (select * from tempemps where deptno is null) t "
-        + "on e.empno = t.empno "
-        + "when matched then update "
-        + "set name = t.name, deptno = t.deptno, salary = t.salary * .1 "
-        + "when not matched then insert (name, dept, salary) "
-        + "values(t.name, 10, t.salary * .15)",
+            + "using (select * from tempemps where deptno is null) t "
+            + "on e.empno = t.empno "
+            + "when matched then update "
+            + "set name = t.name, deptno = t.deptno, salary = t.salary * .1 "
+            + "when not matched then insert (name, dept, salary) "
+            + "values(t.name, 10, t.salary * .15)",
 
         "MERGE INTO `EMPS` AS `E`\n"
-        + "USING (SELECT *\n"
-        + "FROM `TEMPEMPS`\n"
-        + "WHERE (`DEPTNO` IS NULL)) AS `T`\n"
-        + "ON (`E`.`EMPNO` = `T`.`EMPNO`)\n"
-        + "WHEN MATCHED THEN UPDATE SET `NAME` = `T`.`NAME`\n"
-        + ", `DEPTNO` = `T`.`DEPTNO`\n"
-        + ", `SALARY` = (`T`.`SALARY` * 0.1)\n"
-        + "WHEN NOT MATCHED THEN INSERT (`NAME`, `DEPT`, `SALARY`) "
-        + "(VALUES (ROW(`T`.`NAME`, 10, (`T`.`SALARY` * 0.15))))");
+            + "USING (SELECT *\n"
+            + "FROM `TEMPEMPS`\n"
+            + "WHERE (`DEPTNO` IS NULL)) AS `T`\n"
+            + "ON (`E`.`EMPNO` = `T`.`EMPNO`)\n"
+            + "WHEN MATCHED THEN UPDATE SET `NAME` = `T`.`NAME`\n"
+            + ", `DEPTNO` = `T`.`DEPTNO`\n"
+            + ", `SALARY` = (`T`.`SALARY` * 0.1)\n"
+            + "WHEN NOT MATCHED THEN INSERT (`NAME`, `DEPT`, `SALARY`) "
+            + "(VALUES (ROW(`T`.`NAME`, 10, (`T`.`SALARY` * 0.15))))");
   }
 
   @Test public void testMergeTableRefSource() {
     check(
         "merge into emps e "
-        + "using tempemps as t "
-        + "on e.empno = t.empno "
-        + "when matched then update "
-        + "set name = t.name, deptno = t.deptno, salary = t.salary * .1 "
-        + "when not matched then insert (name, dept, salary) "
-        + "values(t.name, 10, t.salary * .15)",
+            + "using tempemps as t "
+            + "on e.empno = t.empno "
+            + "when matched then update "
+            + "set name = t.name, deptno = t.deptno, salary = t.salary * .1 "
+            + "when not matched then insert (name, dept, salary) "
+            + "values(t.name, 10, t.salary * .15)",
 
         "MERGE INTO `EMPS` AS `E`\n"
-        + "USING `TEMPEMPS` AS `T`\n"
-        + "ON (`E`.`EMPNO` = `T`.`EMPNO`)\n"
-        + "WHEN MATCHED THEN UPDATE SET `NAME` = `T`.`NAME`\n"
-        + ", `DEPTNO` = `T`.`DEPTNO`\n"
-        + ", `SALARY` = (`T`.`SALARY` * 0.1)\n"
-        + "WHEN NOT MATCHED THEN INSERT (`NAME`, `DEPT`, `SALARY`) "
-        + "(VALUES (ROW(`T`.`NAME`, 10, (`T`.`SALARY` * 0.15))))");
+            + "USING `TEMPEMPS` AS `T`\n"
+            + "ON (`E`.`EMPNO` = `T`.`EMPNO`)\n"
+            + "WHEN MATCHED THEN UPDATE SET `NAME` = `T`.`NAME`\n"
+            + ", `DEPTNO` = `T`.`DEPTNO`\n"
+            + ", `SALARY` = (`T`.`SALARY` * 0.1)\n"
+            + "WHEN NOT MATCHED THEN INSERT (`NAME`, `DEPT`, `SALARY`) "
+            + "(VALUES (ROW(`T`.`NAME`, 10, (`T`.`SALARY` * 0.15))))");
   }
 
   @Test public void testBitStringNotImplemented() {
@@ -2042,13 +2048,13 @@ public class SqlParserTest {
     checkExp("x'fffff'=X''", "(X'FFFFF' = X'')");
     checkExp(
         "x'1' \t\t\f\r \n"
-        + "'2'--hi this is a comment'FF'\r\r\t\f \n"
-        + "'34'",
+            + "'2'--hi this is a comment'FF'\r\r\t\f \n"
+            + "'34'",
         "X'1'\n'2'\n'34'");
     checkExp(
         "x'1' \t\t\f\r \n"
-        + "'000'--\n"
-        + "'01'",
+            + "'000'--\n"
+            + "'01'",
         "X'1'\n'000'\n'01'");
     checkExp(
         "x'1234567890abcdef'=X'fFeEdDcCbBaA'",
@@ -2073,8 +2079,8 @@ public class SqlParserTest {
     check(
         "select x'1' '2' from t",
         "SELECT X'1'\n"
-        + "'2'\n"
-        + "FROM `T`");
+            + "'2'\n"
+            + "FROM `T`");
   }
 
   @Test public void testStringLiteral() {
@@ -2127,17 +2133,17 @@ public class SqlParserTest {
     check(
         "select N'1' '2' from t",
         "SELECT _ISO-8859-1'1'\n'2'\n"
-        + "FROM `T`");
+            + "FROM `T`");
   }
 
   @Test public void testStringLiteralChain() {
     final String fooBar =
         "'foo'\n"
-        + "'bar'";
+            + "'bar'";
     final String fooBarBaz =
         "'foo'\n"
-        + "'bar'\n"
-        + "'baz'";
+            + "'bar'\n"
+            + "'baz'";
     checkExp("   'foo'\r'bar'", fooBar);
     checkExp("   'foo'\r\n'bar'", fooBar);
     checkExp("   'foo'\r\n\r\n'bar'  \n   'baz'", fooBarBaz);
@@ -2169,19 +2175,19 @@ public class SqlParserTest {
     checkExp(
         "case (select * from emp) when 1 then 2 end",
         "(CASE WHEN ((SELECT *\n"
-        + "FROM `EMP`) = 1) THEN 2 ELSE NULL END)");
+            + "FROM `EMP`) = 1) THEN 2 ELSE NULL END)");
     checkExp(
         "case 1 when (select * from emp) then 2 end",
         "(CASE WHEN (1 = (SELECT *\n"
-        + "FROM `EMP`)) THEN 2 ELSE NULL END)");
+            + "FROM `EMP`)) THEN 2 ELSE NULL END)");
     checkExp(
         "case 1 when 2 then (select * from emp) end",
         "(CASE WHEN (1 = 2) THEN (SELECT *\n"
-        + "FROM `EMP`) ELSE NULL END)");
+            + "FROM `EMP`) ELSE NULL END)");
     checkExp(
         "case 1 when 2 then 3 else (select * from emp) end",
         "(CASE WHEN (1 = 2) THEN 3 ELSE (SELECT *\n"
-        + "FROM `EMP`) END)");
+            + "FROM `EMP`) END)");
     checkExp(
         "case x when 2, 4 then 3 else 4 end",
         "(CASE WHEN (`X` IN (2, 4)) THEN 3 ELSE 4 END)");
@@ -2373,9 +2379,9 @@ public class SqlParserTest {
         "TRIM(TRAILING 'mustache' FROM 'beard')");
     checkExp(
         "trim (coalesce(cast(null as varchar(2)))||"
-        + "' '||coalesce('junk ',''))",
+            + "' '||coalesce('junk ',''))",
         "TRIM(BOTH ' ' FROM ((COALESCE(CAST(NULL AS VARCHAR(2))) || "
-        + "' ') || COALESCE('junk ', '')))");
+            + "' ') || COALESCE('junk ', '')))");
 
     checkFails(
         "trim(^from^ 'beard')",
@@ -2421,9 +2427,9 @@ public class SqlParserTest {
     check(
         "select * from ( select sum(x) over w, sum(y) over w from s window w as (range interval '1' minute preceding))",
         "SELECT *\n"
-        + "FROM (SELECT (SUM(`X`) OVER `W`), (SUM(`Y`) OVER `W`)\n"
-        + "FROM `S`\n"
-        + "WINDOW `W` AS (RANGE INTERVAL '1' MINUTE PRECEDING))");
+            + "FROM (SELECT (SUM(`X`) OVER `W`), (SUM(`Y`) OVER `W`)\n"
+            + "FROM `S`\n"
+            + "WINDOW `W` AS (RANGE INTERVAL '1' MINUTE PRECEDING))");
   }
 
   @Test public void testWindowSpec() {
@@ -2431,14 +2437,14 @@ public class SqlParserTest {
     check(
         "select count(z) over w as foo from Bids window w as (partition by y + yy, yyy order by x rows between 2 preceding and 2 following)",
         "SELECT (COUNT(`Z`) OVER `W`) AS `FOO`\n"
-        + "FROM `BIDS`\n"
-        + "WINDOW `W` AS (PARTITION BY (`Y` + `YY`), `YYY` ORDER BY `X` ROWS BETWEEN 2 PRECEDING AND 2 FOLLOWING)");
+            + "FROM `BIDS`\n"
+            + "WINDOW `W` AS (PARTITION BY (`Y` + `YY`), `YYY` ORDER BY `X` ROWS BETWEEN 2 PRECEDING AND 2 FOLLOWING)");
 
     check(
         "select count(*) over w from emp window w as (rows 2 preceding)",
         "SELECT (COUNT(*) OVER `W`)\n"
-        + "FROM `EMP`\n"
-        + "WINDOW `W` AS (ROWS 2 PRECEDING)");
+            + "FROM `EMP`\n"
+            + "WINDOW `W` AS (ROWS 2 PRECEDING)");
 
     // Chained string literals are valid syntax. They are unlikely to be
     // semantically valid, because intervals are usually numeric or
@@ -2447,16 +2453,16 @@ public class SqlParserTest {
     // since we are just parsing, and not validating the sql.
     check(
         "select count(*) over w from emp window w as (\n"
-        + "  rows 'foo' 'bar'\n"
-        + "       'baz' preceding)",
+            + "  rows 'foo' 'bar'\n"
+            + "       'baz' preceding)",
         "SELECT (COUNT(*) OVER `W`)\n"
-        + "FROM `EMP`\n"
-        + "WINDOW `W` AS (ROWS 'foo'\n'bar'\n'baz' PRECEDING)");
+            + "FROM `EMP`\n"
+            + "WINDOW `W` AS (ROWS 'foo'\n'bar'\n'baz' PRECEDING)");
 
     // Partition clause out of place. Found after ORDER BY
     checkFails(
         "select count(z) over w as foo \n"
-        + "from Bids window w as (partition by y order by x ^partition^ by y)",
+            + "from Bids window w as (partition by y order by x ^partition^ by y)",
         "(?s).*Encountered \"partition\".*");
     checkFails(
         "select count(z) over w as foo from Bids window w as (order by x ^partition^ by y)",
@@ -2490,22 +2496,22 @@ public class SqlParserTest {
     check(
         "select sum(x) over (order by x allow partial) from bids",
         "SELECT (SUM(`X`) OVER (ORDER BY `X`))\n"
-        + "FROM `BIDS`");
+            + "FROM `BIDS`");
 
     check(
         "select sum(x) over (order by x) from bids",
         "SELECT (SUM(`X`) OVER (ORDER BY `X`))\n"
-        + "FROM `BIDS`");
+            + "FROM `BIDS`");
 
     check(
         "select sum(x) over (order by x disallow partial) from bids",
         "SELECT (SUM(`X`) OVER (ORDER BY `X` DISALLOW PARTIAL))\n"
-        + "FROM `BIDS`");
+            + "FROM `BIDS`");
 
     check(
         "select sum(x) over (order by x) from bids",
         "SELECT (SUM(`X`) OVER (ORDER BY `X`))\n"
-        + "FROM `BIDS`");
+            + "FROM `BIDS`");
   }
 
   @Test public void testAs() {
@@ -2513,29 +2519,29 @@ public class SqlParserTest {
     check(
         "select x y from t",
         "SELECT `X` AS `Y`\n"
-        + "FROM `T`");
+            + "FROM `T`");
 
     check(
         "select x AS y from t",
         "SELECT `X` AS `Y`\n"
-        + "FROM `T`");
+            + "FROM `T`");
     check(
         "select sum(x) y from t group by z",
         "SELECT SUM(`X`) AS `Y`\n"
-        + "FROM `T`\n"
-        + "GROUP BY `Z`");
+            + "FROM `T`\n"
+            + "GROUP BY `Z`");
 
     // Even after OVER
     check(
         "select count(z) over w foo from Bids window w as (order by x)",
         "SELECT (COUNT(`Z`) OVER `W`) AS `FOO`\n"
-        + "FROM `BIDS`\n"
-        + "WINDOW `W` AS (ORDER BY `X`)");
+            + "FROM `BIDS`\n"
+            + "WINDOW `W` AS (ORDER BY `X`)");
 
     // AS is optional for table correlation names
     final String expected =
         "SELECT `X`\n"
-        + "FROM `T` AS `T1`";
+            + "FROM `T` AS `T1`";
     check("select x from t as t1", expected);
     check("select x from t t1", expected);
 
@@ -2554,14 +2560,14 @@ public class SqlParserTest {
     check(
         "select x from t as t1 (a, b) where foo",
         "SELECT `X`\n"
-        + "FROM `T` AS `T1` (`A`, `B`)\n"
-        + "WHERE `FOO`");
+            + "FROM `T` AS `T1` (`A`, `B`)\n"
+            + "WHERE `FOO`");
 
     check(
         "select x from (values (1, 2), (3, 4)) as t1 (\"a\", b) where \"a\" > b",
         "SELECT `X`\n"
-        + "FROM (VALUES (ROW(1, 2)), (ROW(3, 4))) AS `T1` (`a`, `B`)\n"
-        + "WHERE (`a` > `B`)");
+            + "FROM (VALUES (ROW(1, 2)), (ROW(3, 4))) AS `T1` (`a`, `B`)\n"
+            + "WHERE (`a` > `B`)");
 
     // must have at least one column
     checkFails(
@@ -2572,15 +2578,15 @@ public class SqlParserTest {
     checkFails(
         "select x from t as t1 (x ^+^ y)",
         "(?s).*Was expecting one of:\n"
-        + "    \"\\)\" \\.\\.\\.\n"
-        + "    \",\" \\.\\.\\..*");
+            + "    \"\\)\" \\.\\.\\.\n"
+            + "    \",\" \\.\\.\\..*");
 
     // cannot have compound identifiers
     checkFails(
         "select x from t as t1 (x^.^y)",
         "(?s).*Was expecting one of:\n"
-        + "    \"\\)\" \\.\\.\\.\n"
-        + "    \",\" \\.\\.\\..*");
+            + "    \"\\)\" \\.\\.\\.\n"
+            + "    \",\" \\.\\.\\..*");
   }
 
   @Test public void testOver() {
@@ -2664,7 +2670,7 @@ public class SqlParserTest {
     checkExp(
         "multiset(select*from T)",
         "(MULTISET ((SELECT *\n"
-        + "FROM `T`)))");
+            + "FROM `T`)))");
   }
 
   @Test public void testMultisetUnion() {
@@ -4900,22 +4906,22 @@ public class SqlParserTest {
     checkExpFails(
         "interval '1^'^",
         "Encountered \"<EOF>\" at line 1, column 12\\.\n"
-        + "Was expecting one of:\n"
-        + "    \"YEAR\" \\.\\.\\.\n"
-        + "    \"MONTH\" \\.\\.\\.\n"
-        + "    \"DAY\" \\.\\.\\.\n"
-        + "    \"HOUR\" \\.\\.\\.\n"
-        + "    \"MINUTE\" \\.\\.\\.\n"
-        + "    \"SECOND\" \\.\\.\\.\n"
-        + "    ");
+            + "Was expecting one of:\n"
+            + "    \"YEAR\" \\.\\.\\.\n"
+            + "    \"MONTH\" \\.\\.\\.\n"
+            + "    \"DAY\" \\.\\.\\.\n"
+            + "    \"HOUR\" \\.\\.\\.\n"
+            + "    \"MINUTE\" \\.\\.\\.\n"
+            + "    \"SECOND\" \\.\\.\\.\n"
+            + "    ");
 
     // illegal qualifiers, no precision in either field
     checkExpFails(
         "interval '1' year ^to^ year",
         "(?s)Encountered \"to year\" at line 1, column 19.\n"
-        + "Was expecting one of:\n"
-        + "    <EOF> \n"
-        + "    \"NOT\" \\.\\.\\..*");
+            + "Was expecting one of:\n"
+            + "    <EOF> \n"
+            + "    \"NOT\" \\.\\.\\..*");
     checkExpFails("interval '1-2' year ^to^ day", ANY);
     checkExpFails("interval '1-2' year ^to^ hour", ANY);
     checkExpFails("interval '1-2' year ^to^ minute", ANY);
@@ -5255,11 +5261,11 @@ public class SqlParserTest {
     check(
         "select*from unnest(x)",
         "SELECT *\n"
-        + "FROM (UNNEST(`X`))");
+            + "FROM (UNNEST(`X`))");
     check(
         "select*from unnest(x) AS T",
         "SELECT *\n"
-        + "FROM (UNNEST(`X`)) AS `T`");
+            + "FROM (UNNEST(`X`)) AS `T`");
 
     // UNNEST cannot be first word in query
     checkFails(
@@ -5374,7 +5380,7 @@ public class SqlParserTest {
     check(
         "SELECT *\n\tFROM mytable",
         "SELECT *\n"
-        + "FROM `MYTABLE`");
+            + "FROM `MYTABLE`");
 
     // make sure that the tab stops do not affect the placement of the
     // error tokens
@@ -5396,26 +5402,26 @@ public class SqlParserTest {
     check(
         "select * from " + ident128,
         "SELECT *\n"
-        + "FROM `" + ident128Upper + "`");
+            + "FROM `" + ident128Upper + "`");
     checkFails(
         "select * from ^" + ident129 + "^",
         "Length of identifier '" + ident129Upper
-        + "' must be less than or equal to 128 characters");
+            + "' must be less than or equal to 128 characters");
 
     check(
         "select " + ident128 + " from mytable",
         "SELECT `" + ident128Upper + "`\n"
-        + "FROM `MYTABLE`");
+            + "FROM `MYTABLE`");
     checkFails(
         "select ^" + ident129 + "^ from mytable",
         "Length of identifier '" + ident129Upper
-        + "' must be less than or equal to 128 characters");
+            + "' must be less than or equal to 128 characters");
   }
 
   /**
    * Tests that you can't quote the names of builtin functions.
    *
-   * @see org.eigenbase.test.SqlValidatorTest#testQuotedFunction()
+   * @see org.apache.calcite.test.SqlValidatorTest#testQuotedFunction()
    */
   @Test public void testQuotedFunction() {
     checkExpFails(
@@ -5439,19 +5445,19 @@ public class SqlParserTest {
     // parser, the literal already contains Unicode characters.
     String in1 =
         "values _UTF16'"
-        + ConversionUtil.TEST_UNICODE_STRING + "'";
+            + ConversionUtil.TEST_UNICODE_STRING + "'";
     String out1 =
         "(VALUES (ROW(_UTF16'"
-        + ConversionUtil.TEST_UNICODE_STRING + "')))";
+            + ConversionUtil.TEST_UNICODE_STRING + "')))";
     check(in1, out1);
 
     // Without the U& prefix, escapes are left unprocessed
     String in2 =
         "values '"
-        + ConversionUtil.TEST_UNICODE_SQL_ESCAPED_LITERAL + "'";
+            + ConversionUtil.TEST_UNICODE_SQL_ESCAPED_LITERAL + "'";
     String out2 =
         "(VALUES (ROW('"
-        + ConversionUtil.TEST_UNICODE_SQL_ESCAPED_LITERAL + "')))";
+            + ConversionUtil.TEST_UNICODE_SQL_ESCAPED_LITERAL + "')))";
     check(in2, out2);
 
     // Likewise, even with the U& prefix, if some other escape
@@ -5459,11 +5465,11 @@ public class SqlParserTest {
     // sequences are not interpreted
     String in3 =
         "values U&'"
-        + ConversionUtil.TEST_UNICODE_SQL_ESCAPED_LITERAL
-        + "' UESCAPE '!'";
+            + ConversionUtil.TEST_UNICODE_SQL_ESCAPED_LITERAL
+            + "' UESCAPE '!'";
     String out3 =
         "(VALUES (ROW(_UTF16'"
-        + ConversionUtil.TEST_UNICODE_SQL_ESCAPED_LITERAL + "')))";
+            + ConversionUtil.TEST_UNICODE_SQL_ESCAPED_LITERAL + "')))";
     check(in3, out3);
   }
 
@@ -5473,10 +5479,10 @@ public class SqlParserTest {
     // by the SQL parser.
     String in =
         "values U&'"
-        + ConversionUtil.TEST_UNICODE_SQL_ESCAPED_LITERAL + "'";
+            + ConversionUtil.TEST_UNICODE_SQL_ESCAPED_LITERAL + "'";
     String out =
         "(VALUES (ROW(_UTF16'"
-        + ConversionUtil.TEST_UNICODE_STRING + "')))";
+            + ConversionUtil.TEST_UNICODE_STRING + "')))";
     check(in, out);
 
     // Verify that we can override with an explicit escape character
@@ -5525,9 +5531,9 @@ public class SqlParserTest {
     SqlSetOption opt = (SqlSetOption) node;
     assertThat(opt.getScope(), equalTo("SYSTEM"));
     assertThat(opt.getName(), equalTo("SCHEMA"));
-    SqlPrettyWriter writer = new SqlPrettyWriter(SqlDialect.EIGENBASE);
+    SqlPrettyWriter writer = new SqlPrettyWriter(SqlDialect.CALCITE);
     assertThat(writer.format(opt.getValue()), equalTo("TRUE"));
-    writer = new SqlPrettyWriter(SqlDialect.EIGENBASE);
+    writer = new SqlPrettyWriter(SqlDialect.CALCITE);
     assertThat(writer.format(opt),
         equalTo("ALTER SYSTEM SET \"SCHEMA\" = TRUE"));
 
@@ -5593,8 +5599,7 @@ public class SqlParserTest {
         sqlNode = parseStmt(sql);
       } catch (SqlParseException e) {
         e.printStackTrace();
-        String message =
-            "Received error while parsing SQL '" + sql
+        String message = "Received error while parsing SQL '" + sql
             + "'; error is:\n"
             + e.toString();
         throw new AssertionError(message);
@@ -5618,8 +5623,7 @@ public class SqlParserTest {
       try {
         sqlNode = parseExpression(sql);
       } catch (SqlParseException e) {
-        String message =
-            "Received error while parsing SQL '" + sql
+        String message = "Received error while parsing SQL '" + sql
             + "'; error is:\n"
             + e.toString();
         throw new RuntimeException(message, e);
@@ -5677,12 +5681,12 @@ public class SqlParserTest {
       // Unparse again in Eigenbase dialect (which we can parse), and
       // minimal parentheses.
       final String sql1 =
-          sqlNode.toSqlString(SqlDialect.EIGENBASE, false).getSql();
+          sqlNode.toSqlString(SqlDialect.CALCITE, false).getSql();
 
       // Parse and unparse again.
       SqlNode sqlNode2 = parseStmtAndHandleEx(sql1);
       final String sql2 =
-          sqlNode2.toSqlString(SqlDialect.EIGENBASE, false).getSql();
+          sqlNode2.toSqlString(SqlDialect.CALCITE, false).getSql();
 
       // Should be the same as we started with.
       assertEquals(sql1, sql2);
@@ -5704,12 +5708,12 @@ public class SqlParserTest {
       // Unparse again in Eigenbase dialect (which we can parse), and
       // minimal parentheses.
       final String sql1 =
-          sqlNode.toSqlString(SqlDialect.EIGENBASE, false).getSql();
+          sqlNode.toSqlString(SqlDialect.CALCITE, false).getSql();
 
       // Parse and unparse again.
       SqlNode sqlNode2 = parseExpressionAndHandleEx(sql1);
       final String sql2 =
-          sqlNode2.toSqlString(SqlDialect.EIGENBASE, false).getSql();
+          sqlNode2.toSqlString(SqlDialect.CALCITE, false).getSql();
 
       // Should be the same as we started with.
       assertEquals(sql1, sql2);

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/sql/parser/SqlUnParserTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/parser/SqlUnParserTest.java b/core/src/test/java/org/apache/calcite/sql/parser/SqlUnParserTest.java
index 163dab8..4713f5f 100644
--- a/core/src/test/java/org/apache/calcite/sql/parser/SqlUnParserTest.java
+++ b/core/src/test/java/org/apache/calcite/sql/parser/SqlUnParserTest.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.parser;
+package org.apache.calcite.sql.parser;
 
 /**
  * Extension to {@link SqlParserTest} which ensures that every expression can

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/sql/test/DefaultSqlTestFactory.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/test/DefaultSqlTestFactory.java b/core/src/test/java/org/apache/calcite/sql/test/DefaultSqlTestFactory.java
index c2538f5..16a5391 100644
--- a/core/src/test/java/org/apache/calcite/sql/test/DefaultSqlTestFactory.java
+++ b/core/src/test/java/org/apache/calcite/sql/test/DefaultSqlTestFactory.java
@@ -14,22 +14,24 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.test;
+package org.apache.calcite.sql.test;
 
-import org.eigenbase.reltype.RelDataTypeFactory;
-import org.eigenbase.reltype.RelDataTypeSystem;
-import org.eigenbase.sql.SqlOperatorTable;
-import org.eigenbase.sql.advise.SqlAdvisor;
-import org.eigenbase.sql.fun.SqlStdOperatorTable;
-import org.eigenbase.sql.parser.SqlParser;
-import org.eigenbase.sql.parser.impl.SqlParserImpl;
-import org.eigenbase.sql.type.SqlTypeFactoryImpl;
-import org.eigenbase.sql.validate.*;
-import org.eigenbase.test.MockCatalogReader;
-import org.eigenbase.test.MockSqlOperatorTable;
-
-import net.hydromatic.avatica.Casing;
-import net.hydromatic.avatica.Quoting;
+import org.apache.calcite.avatica.Casing;
+import org.apache.calcite.avatica.Quoting;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.sql.SqlOperatorTable;
+import org.apache.calcite.sql.advise.SqlAdvisor;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.parser.SqlParser;
+import org.apache.calcite.sql.parser.impl.SqlParserImpl;
+import org.apache.calcite.sql.type.SqlTypeFactoryImpl;
+import org.apache.calcite.sql.validate.SqlConformance;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql.validate.SqlValidatorUtil;
+import org.apache.calcite.sql.validate.SqlValidatorWithHints;
+import org.apache.calcite.test.MockCatalogReader;
+import org.apache.calcite.test.MockSqlOperatorTable;
 
 import com.google.common.collect.ImmutableMap;
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/sql/test/DelegatingSqlTestFactory.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/test/DelegatingSqlTestFactory.java b/core/src/test/java/org/apache/calcite/sql/test/DelegatingSqlTestFactory.java
index 82a1ffc..211f1f4 100644
--- a/core/src/test/java/org/apache/calcite/sql/test/DelegatingSqlTestFactory.java
+++ b/core/src/test/java/org/apache/calcite/sql/test/DelegatingSqlTestFactory.java
@@ -14,25 +14,26 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.test;
+package org.apache.calcite.sql.test;
 
-import org.eigenbase.sql.SqlOperatorTable;
-import org.eigenbase.sql.advise.SqlAdvisor;
-import org.eigenbase.sql.parser.SqlParser;
-import org.eigenbase.sql.validate.SqlValidator;
-import org.eigenbase.sql.validate.SqlValidatorWithHints;
+import org.apache.calcite.sql.SqlOperatorTable;
+import org.apache.calcite.sql.advise.SqlAdvisor;
+import org.apache.calcite.sql.parser.SqlParser;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql.validate.SqlValidatorWithHints;
 
 /**
 * Implementation of {@link SqlTestFactory} that delegates
  * everything to an underlying factory.
  *
  * <p>Generally a chain starts with a
- * {@link org.eigenbase.sql.test.DefaultSqlTestFactory}, and continues with a
- * succession of objects that derive from {@code DelegatingSqlTestFactory}
- * and override one method.</p>
+ * {@link org.apache.calcite.sql.test.DefaultSqlTestFactory}, and continues with
+ * a succession of objects that derive from {@code DelegatingSqlTestFactory} and
+ * override one method.</p>
  *
- * <p>Methods such as {@link org.eigenbase.sql.test.SqlTester#withConformance}
- * help create such chains.</p>
+ * <p>Methods such as
+ * {@link org.apache.calcite.sql.test.SqlTester#withConformance} help create
+ * such chains.</p>
 */
 public class DelegatingSqlTestFactory implements SqlTestFactory {
   private final SqlTestFactory factory;


[24/58] [abbrv] [partial] incubator-calcite git commit: [CALCITE-306] Standardize code style for "import package.*; "

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/type/OrdinalReturnTypeInference.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/OrdinalReturnTypeInference.java b/core/src/main/java/org/apache/calcite/sql/type/OrdinalReturnTypeInference.java
index 08bfad4..b5a6f97 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/OrdinalReturnTypeInference.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/OrdinalReturnTypeInference.java
@@ -14,11 +14,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.type;
+package org.apache.calcite.sql.type;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.validate.*;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlOperatorBinding;
 
 /**
  * Returns the type of the operand at a particular 0-based ordinal position.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 55c2893..219cb99 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
@@ -14,16 +14,25 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.type;
+package org.apache.calcite.sql.type;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rel.type.RelDataTypeImpl;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.rel.type.RelProtoDataType;
+import org.apache.calcite.sql.ExplicitOperatorBinding;
+import org.apache.calcite.sql.SqlCallBinding;
+import org.apache.calcite.sql.SqlCollation;
+import org.apache.calcite.sql.SqlOperatorBinding;
+import org.apache.calcite.sql.SqlUtil;
+import org.apache.calcite.util.Util;
 
 import java.util.AbstractList;
 import java.util.List;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.util.Util;
-
-import static org.eigenbase.util.Static.RESOURCE;
+import static org.apache.calcite.util.Static.RESOURCE;
 
 /**
  * A collection of return-type inference strategies.
@@ -118,8 +127,8 @@ public abstract class ReturnTypes {
    */
   public static final SqlReturnTypeInference ARG0_NULLABLE_IF_EMPTY =
       new OrdinalReturnTypeInference(0) {
-        @Override
-        public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
+        @Override public RelDataType
+        inferReturnType(SqlOperatorBinding opBinding) {
           final RelDataType type = super.inferReturnType(opBinding);
           if (opBinding.getGroupCount() == 0) {
             return opBinding.getTypeFactory()
@@ -388,7 +397,7 @@ public abstract class ReturnTypes {
   /**
    * Same as {@link #DECIMAL_PRODUCT} but returns with nullability if any of
    * the operands is nullable by using
-   * {@link org.eigenbase.sql.type.SqlTypeTransforms#TO_NULLABLE}
+   * {@link org.apache.calcite.sql.type.SqlTypeTransforms#TO_NULLABLE}
    */
   public static final SqlReturnTypeInference DECIMAL_PRODUCT_NULLABLE =
       cascade(DECIMAL_PRODUCT, SqlTypeTransforms.TO_NULLABLE);
@@ -422,7 +431,7 @@ public abstract class ReturnTypes {
   /**
    * Same as {@link #DECIMAL_QUOTIENT} but returns with nullability if any of
    * the operands is nullable by using
-   * {@link org.eigenbase.sql.type.SqlTypeTransforms#TO_NULLABLE}
+   * {@link org.apache.calcite.sql.type.SqlTypeTransforms#TO_NULLABLE}
    */
   public static final SqlReturnTypeInference DECIMAL_QUOTIENT_NULLABLE =
       cascade(DECIMAL_QUOTIENT, SqlTypeTransforms.TO_NULLABLE);
@@ -492,14 +501,14 @@ public abstract class ReturnTypes {
   /**
    * Same as {@link #DECIMAL_SUM} but returns with nullability if any
    * of the operands is nullable by using
-   * {@link org.eigenbase.sql.type.SqlTypeTransforms#TO_NULLABLE}.
+   * {@link org.apache.calcite.sql.type.SqlTypeTransforms#TO_NULLABLE}.
    */
   public static final SqlReturnTypeInference DECIMAL_SUM_NULLABLE =
       cascade(DECIMAL_SUM, SqlTypeTransforms.TO_NULLABLE);
 
   /**
-   * Type-inference strategy whereby the result type of a call is {@link
-   * #DECIMAL_SUM_NULLABLE} with a fallback to {@link #LEAST_RESTRICTIVE}
+   * Type-inference strategy whereby the result type of a call is
+   * {@link #DECIMAL_SUM_NULLABLE} with a fallback to {@link #LEAST_RESTRICTIVE}
    * These rules are used for addition and subtraction.
    */
   public static final SqlReturnTypeInference NULLABLE_SUM =
@@ -589,8 +598,8 @@ public abstract class ReturnTypes {
       };
   /**
    * Same as {@link #DYADIC_STRING_SUM_PRECISION} and using
-   * {@link org.eigenbase.sql.type.SqlTypeTransforms#TO_NULLABLE},
-   * {@link org.eigenbase.sql.type.SqlTypeTransforms#TO_VARYING}.
+   * {@link org.apache.calcite.sql.type.SqlTypeTransforms#TO_NULLABLE},
+   * {@link org.apache.calcite.sql.type.SqlTypeTransforms#TO_VARYING}.
    */
   public static final SqlReturnTypeInference
   DYADIC_STRING_SUM_PRECISION_NULLABLE_VARYING =
@@ -598,8 +607,8 @@ public abstract class ReturnTypes {
           SqlTypeTransforms.TO_VARYING);
 
   /**
-   * Same as {@link #DYADIC_STRING_SUM_PRECISION} and using {@link
-   * org.eigenbase.sql.type.SqlTypeTransforms#TO_NULLABLE}
+   * Same as {@link #DYADIC_STRING_SUM_PRECISION} and using
+   * {@link org.apache.calcite.sql.type.SqlTypeTransforms#TO_NULLABLE}
    */
   public static final SqlReturnTypeInference
   DYADIC_STRING_SUM_PRECISION_NULLABLE =
@@ -607,7 +616,7 @@ public abstract class ReturnTypes {
 
   /**
    * Type-inference strategy where the expression is assumed to be registered
-   * as a {@link org.eigenbase.sql.validate.SqlValidatorNamespace}, and
+   * as a {@link org.apache.calcite.sql.validate.SqlValidatorNamespace}, and
    * therefore the result type of the call is the type of that namespace.
    */
   public static final SqlReturnTypeInference SCOPE =

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/type/SameOperandTypeChecker.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SameOperandTypeChecker.java b/core/src/main/java/org/apache/calcite/sql/type/SameOperandTypeChecker.java
index 7c84ec1..5f853b8 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/SameOperandTypeChecker.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/SameOperandTypeChecker.java
@@ -14,17 +14,23 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.type;
+package org.apache.calcite.sql.type;
 
-import java.util.*;
-
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.util.Util;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlCallBinding;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperandCountRange;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlOperatorBinding;
+import org.apache.calcite.sql.SqlUtil;
+import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
 
-import static org.eigenbase.util.Static.RESOURCE;
+import java.util.Collections;
+import java.util.List;
+
+import static org.apache.calcite.util.Static.RESOURCE;
 
 /**
  * Parameter type-checking strategy where all operand types must be the same.
@@ -103,8 +109,9 @@ public class SameOperandTypeChecker implements SqlSingleOperandTypeChecker {
   }
 
   /**
-   * Similar functionality to {@link #checkOperandTypes(SqlCallBinding,
-   * boolean)}, but not part of the interface, and cannot throw an error.
+   * Similar functionality to
+   * {@link #checkOperandTypes(SqlCallBinding, boolean)}, but not part of the
+   * interface, and cannot throw an error.
    */
   public boolean checkOperandTypes(
       SqlOperatorBinding operatorBinding) {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/type/SetopOperandTypeChecker.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SetopOperandTypeChecker.java b/core/src/main/java/org/apache/calcite/sql/type/SetopOperandTypeChecker.java
index 5ea2dcd..dcf89f4 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/SetopOperandTypeChecker.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/SetopOperandTypeChecker.java
@@ -14,16 +14,23 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.type;
+package org.apache.calcite.sql.type;
 
-import java.util.*;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.sql.SqlCallBinding;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperandCountRange;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlSelect;
+import org.apache.calcite.sql.SqlUtil;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.util.Util;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.validate.*;
-import org.eigenbase.util.*;
+import java.util.AbstractList;
+import java.util.List;
 
-import static org.eigenbase.util.Static.RESOURCE;
+import static org.apache.calcite.util.Static.RESOURCE;
 
 /**
  * Parameter type-checking strategy for a set operator (UNION, INTERSECT,

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/type/SqlOperandCountRanges.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SqlOperandCountRanges.java b/core/src/main/java/org/apache/calcite/sql/type/SqlOperandCountRanges.java
index 4858c06..1420c8a 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/SqlOperandCountRanges.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/SqlOperandCountRanges.java
@@ -14,9 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.type;
+package org.apache.calcite.sql.type;
 
-import org.eigenbase.sql.SqlOperandCountRange;
+import org.apache.calcite.sql.SqlOperandCountRange;
 
 /**
  * Helpers for {@link SqlOperandCountRange}.
@@ -39,6 +39,7 @@ public abstract class SqlOperandCountRanges {
     return new RangeImpl(0, -1);
   }
 
+  /** Implementation of {@link SqlOperandCountRange}. */
   private static class RangeImpl implements SqlOperandCountRange {
     private final int min;
     private final int max;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/type/SqlOperandTypeChecker.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SqlOperandTypeChecker.java b/core/src/main/java/org/apache/calcite/sql/type/SqlOperandTypeChecker.java
index 8cc4787..b6a21f6 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/SqlOperandTypeChecker.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/SqlOperandTypeChecker.java
@@ -14,15 +14,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.type;
+package org.apache.calcite.sql.type;
 
-import org.eigenbase.sql.*;
+import org.apache.calcite.sql.SqlCallBinding;
+import org.apache.calcite.sql.SqlOperandCountRange;
+import org.apache.calcite.sql.SqlOperator;
 
 /**
  * Strategy interface to check for allowed operand types of an operator call.
  *
- * <p>This interface is an example of the {@link
- * org.eigenbase.util.Glossary#STRATEGY_PATTERN strategy pattern}.</p>
+ * <p>This interface is an example of the
+ * {@link org.apache.calcite.util.Glossary#STRATEGY_PATTERN strategy pattern}.
  */
 public interface SqlOperandTypeChecker {
   //~ Methods ----------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/type/SqlOperandTypeInference.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SqlOperandTypeInference.java b/core/src/main/java/org/apache/calcite/sql/type/SqlOperandTypeInference.java
index 75d4ab9..8d55188 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/SqlOperandTypeInference.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/SqlOperandTypeInference.java
@@ -14,10 +14,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.type;
+package org.apache.calcite.sql.type;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlCallBinding;
 
 /**
  * Strategy to infer unknown types of the operands of an operator call.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/type/SqlReturnTypeInference.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SqlReturnTypeInference.java b/core/src/main/java/org/apache/calcite/sql/type/SqlReturnTypeInference.java
index 2f2b686..85b8717 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/SqlReturnTypeInference.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/SqlReturnTypeInference.java
@@ -14,17 +14,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.type;
+package org.apache.calcite.sql.type;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlOperatorBinding;
 
 /**
  * Strategy interface to infer the type of an operator call from the type of the
  * operands.
  *
- * <p>This interface is an example of the {@link
- * org.eigenbase.util.Glossary#STRATEGY_PATTERN strategy pattern}. This makes
+ * <p>This interface is an example of the
+ * {@link org.apache.calcite.util.Glossary#STRATEGY_PATTERN strategy pattern}.
+ * This makes
  * sense because many operators have similar, straightforward strategies, such
  * as to take the type of the first operand.</p>
  */

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/type/SqlReturnTypeInferenceChain.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SqlReturnTypeInferenceChain.java b/core/src/main/java/org/apache/calcite/sql/type/SqlReturnTypeInferenceChain.java
index 45be2bd..7f04f6e 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/SqlReturnTypeInferenceChain.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/SqlReturnTypeInferenceChain.java
@@ -14,10 +14,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.type;
+package org.apache.calcite.sql.type;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlOperatorBinding;
 
 import com.google.common.collect.ImmutableList;
 
@@ -38,7 +38,7 @@ public class SqlReturnTypeInferenceChain implements SqlReturnTypeInference {
    * Creates a SqlReturnTypeInferenceChain from an array of rules.
    *
    * <p>Package-protected.
-   * Use {@link org.eigenbase.sql.type.ReturnTypes#chain}.</p>
+   * Use {@link org.apache.calcite.sql.type.ReturnTypes#chain}.</p>
    */
   SqlReturnTypeInferenceChain(SqlReturnTypeInference... rules) {
     assert rules != null;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/type/SqlSingleOperandTypeChecker.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SqlSingleOperandTypeChecker.java b/core/src/main/java/org/apache/calcite/sql/type/SqlSingleOperandTypeChecker.java
index 6529f68..e1b4b09 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/SqlSingleOperandTypeChecker.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/SqlSingleOperandTypeChecker.java
@@ -14,9 +14,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.type;
+package org.apache.calcite.sql.type;
 
-import org.eigenbase.sql.*;
+import org.apache.calcite.sql.SqlCallBinding;
+import org.apache.calcite.sql.SqlNode;
 
 /**
  * SqlSingleOperandTypeChecker is an extension of {@link SqlOperandTypeChecker}

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 bf8fc42..8ed26b8 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
@@ -14,11 +14,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.type;
+package org.apache.calcite.sql.type;
 
-import java.util.*;
+import org.apache.calcite.util.Util;
 
-import org.eigenbase.util.*;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
 
 /**
  * Class to hold rules to determine if a type is assignable from another type.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/type/SqlTypeExplicitPrecedenceList.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeExplicitPrecedenceList.java b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeExplicitPrecedenceList.java
index 2469707..1da123d 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeExplicitPrecedenceList.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeExplicitPrecedenceList.java
@@ -14,19 +14,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.type;
+package org.apache.calcite.sql.type;
 
-import java.util.*;
-
-import org.eigenbase.reltype.*;
-import org.eigenbase.util.ImmutableNullableList;
-import org.eigenbase.util.Util;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypePrecedenceList;
+import org.apache.calcite.util.ImmutableNullableList;
+import org.apache.calcite.util.Util;
 
 import com.google.common.base.Predicates;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Iterables;
 
+import java.util.List;
+import java.util.Map;
+
 /**
  * SqlTypeExplicitPrecedenceList implements the
  * {@link RelDataTypePrecedenceList} interface via an explicit list of

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/type/SqlTypeFactoryImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeFactoryImpl.java b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeFactoryImpl.java
index 04110b8..3003269 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeFactoryImpl.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeFactoryImpl.java
@@ -14,15 +14,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.type;
-
-import java.nio.charset.*;
+package org.apache.calcite.sql.type;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeFactoryImpl;
+import org.apache.calcite.rel.type.RelDataTypeFamily;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.sql.SqlCollation;
+import org.apache.calcite.sql.SqlIntervalQualifier;
+import org.apache.calcite.util.Util;
+
+import java.nio.charset.Charset;
 import java.util.List;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.util.*;
-
 /**
  * SqlTypeFactoryImpl provides a default implementation of
  * {@link RelDataTypeFactory} which supports SQL types.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/type/SqlTypeFamily.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeFamily.java b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeFamily.java
index 5c9ff55..ce3fd6e 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeFamily.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeFamily.java
@@ -14,16 +14,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.type;
+package org.apache.calcite.sql.type;
 
-import java.sql.*;
-import java.util.*;
-
-import org.eigenbase.reltype.*;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFamily;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 
+import java.sql.Types;
+import java.util.List;
+import java.util.Map;
+
 /**
  * SqlTypeFamily provides SQL type categorization.
  *

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/type/SqlTypeName.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeName.java b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeName.java
index fd6603e..0391923 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeName.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeName.java
@@ -14,23 +14,24 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.type;
+package org.apache.calcite.sql.type;
 
-import java.math.*;
-
-import java.sql.*;
-
-import java.util.*;
-
-import org.eigenbase.reltype.RelDataTypeSystem;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.util.*;
-import org.eigenbase.util14.DateTimeUtil;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.util.DateTimeUtil;
+import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 
+import java.math.BigDecimal;
+import java.sql.Types;
+import java.util.Arrays;
+import java.util.Calendar;
+import java.util.List;
+import java.util.Map;
+
 /**
  * Enumeration of the type names which can be used to construct a SQL type.
  * Rationale for this class's existence (instead of just using the standard
@@ -302,7 +303,7 @@ public enum SqlTypeName {
    * precision is either unsupported or must be specified explicitly.
    *
    * @deprecated Use
-   * {@link org.eigenbase.reltype.RelDataTypeSystem#getDefaultPrecision(SqlTypeName)};
+   * {@link org.apache.calcite.rel.type.RelDataTypeSystem#getDefaultPrecision(SqlTypeName)};
    * will be removed after calcite-0.9.1.
    */
   public int getDefaultPrecision() {
@@ -681,7 +682,7 @@ public enum SqlTypeName {
    * @return Maximum allowed precision
    *
    * @deprecated Use
-   * {@link org.eigenbase.reltype.RelDataTypeSystem#getMaxScale(SqlTypeName)};
+   * {@link org.apache.calcite.rel.type.RelDataTypeSystem#getMaxScale(SqlTypeName)};
    * will be removed after calcite-0.9.1.
    */
   public int getMaxPrecision() {
@@ -696,7 +697,7 @@ public enum SqlTypeName {
    * @return Maximum allowed scale
    *
    * @deprecated Use
-   * {@link org.eigenbase.reltype.RelDataTypeSystem#getMaxScale(SqlTypeName)};
+   * {@link org.apache.calcite.rel.type.RelDataTypeSystem#getMaxScale(SqlTypeName)};
    * will be removed after calcite-0.9.1.
    */
   public int getMaxScale() {
@@ -745,6 +746,7 @@ public enum SqlTypeName {
     }
   }
 
+  /** Limit. */
   public enum Limit {
     ZERO, UNDERFLOW, OVERFLOW
   }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/type/SqlTypeTransform.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeTransform.java b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeTransform.java
index 3413c22..5d413e4 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeTransform.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeTransform.java
@@ -14,10 +14,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.type;
+package org.apache.calcite.sql.type;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlOperatorBinding;
 
 /**
  * Strategy to transform one type to another. The transformation is dependent on
@@ -25,8 +25,8 @@ import org.eigenbase.sql.*;
  * type and the other operands. Can not be used by itself. Must be used in an
  * object of type {@link SqlTypeTransformCascade}.
  *
- * <p>This class is an example of the {@link
- * org.eigenbase.util.Glossary#STRATEGY_PATTERN strategy pattern}.</p>
+ * <p>This class is an example of the
+ * {@link org.apache.calcite.util.Glossary#STRATEGY_PATTERN strategy pattern}.
  */
 public interface SqlTypeTransform {
   //~ Methods ----------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/type/SqlTypeTransformCascade.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeTransformCascade.java b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeTransformCascade.java
index 1731e5b..5dfc7e4 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeTransformCascade.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeTransformCascade.java
@@ -14,17 +14,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.type;
+package org.apache.calcite.sql.type;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlOperatorBinding;
 
 import com.google.common.collect.ImmutableList;
 
 /**
  * Strategy to infer the type of an operator call from the type of the operands
- * by using one {@link SqlReturnTypeInference} rule and a combination of {@link
- * SqlTypeTransform}s
+ * by using one {@link SqlReturnTypeInference} rule and a combination of
+ * {@link SqlTypeTransform}s
  */
 public class SqlTypeTransformCascade implements SqlReturnTypeInference {
   //~ Instance fields --------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/type/SqlTypeTransforms.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeTransforms.java b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeTransforms.java
index c4909b7..89ab33b 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeTransforms.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeTransforms.java
@@ -14,19 +14,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.type;
+package org.apache.calcite.sql.type;
 
-import java.util.List;
-
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.util.*;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.sql.SqlOperatorBinding;
+import org.apache.calcite.util.Util;
 
 import com.google.common.base.Preconditions;
 
+import java.util.List;
+
 /**
- * SqlTypeTransforms defines a number of reusable instances of {@link
- * SqlTypeTransform}.
+ * SqlTypeTransforms defines a number of reusable instances of
+ * {@link SqlTypeTransform}.
  *
  * <p>NOTE: avoid anonymous inner classes here except for unique,
  * non-generalizable strategies; anything else belongs in a reusable top-level

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 2afc8e1..63f8584 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
@@ -14,23 +14,37 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.type;
-
-import java.nio.charset.*;
-import java.util.*;
-
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.sql.validate.*;
-import org.eigenbase.util.*;
-import org.eigenbase.util14.*;
+package org.apache.calcite.sql.type;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeFamily;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rel.type.RelDataTypeFieldImpl;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlCallBinding;
+import org.apache.calcite.sql.SqlCollation;
+import org.apache.calcite.sql.SqlDataTypeSpec;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.parser.SqlParserPos;
+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.NumberUtil;
+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 com.google.common.collect.Lists;
 
-import static org.eigenbase.util.Static.RESOURCE;
+import java.nio.charset.Charset;
+import java.util.AbstractList;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.calcite.util.Static.RESOURCE;
 
 /**
  * Contains utility methods used during SQL validation or type derivation.
@@ -65,16 +79,14 @@ public abstract class SqlTypeUtil {
       }
 
       if (t0.getCharset() == null) {
-        throw Util.newInternal(
-            "RelDataType object should have been assigned a "
+        throw Util.newInternal("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 "
+        throw Util.newInternal("RelDataType object should have been assigned a "
             + "(default) collation when calling deriveType");
       } else if (!t0.getCollation().getCharset().equals(
           t1.getCollation().getCharset())) {
@@ -718,10 +730,10 @@ public abstract class SqlTypeUtil {
    * Compares two types and returns true if fromType can be cast to toType.
    *
    * <p>REVIEW jvs 17-Dec-2004: the coerce param below shouldn't really be
-   * necessary. We're using it as a hack because {@link
-   * SqlTypeFactoryImpl#leastRestrictiveSqlType} isn't complete enough yet.
-   * Once it is, this param (and the non-coerce rules of {@link
-   * SqlTypeAssignmentRules}) should go away.
+   * necessary. We're using it as a hack because
+   * {@link SqlTypeFactoryImpl#leastRestrictiveSqlType} isn't complete enough
+   * yet.  Once it is, this param (and the non-coerce rules of
+   * {@link SqlTypeAssignmentRules}) should go away.
    *
    * @param toType   target of assignment
    * @param fromType source of assignment
@@ -1116,13 +1128,11 @@ public abstract class SqlTypeUtil {
     final List<RelDataTypeField> fields = rowType.getFieldList();
 
     return new AbstractList<RelDataType>() {
-      @Override
-      public RelDataType get(int index) {
+      @Override public RelDataType get(int index) {
         return fields.get(requiredFields.get(index).intValue()).getType();
       }
 
-      @Override
-      public int size() {
+      @Override public int size() {
         return requiredFields.size();
       }
     };

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/type/TableFunctionReturnTypeInference.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/TableFunctionReturnTypeInference.java b/core/src/main/java/org/apache/calcite/sql/type/TableFunctionReturnTypeInference.java
index 52a2004..7702c03 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/TableFunctionReturnTypeInference.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/TableFunctionReturnTypeInference.java
@@ -14,15 +14,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.type;
+package org.apache.calcite.sql.type;
 
-import java.util.*;
+import org.apache.calcite.rel.metadata.RelColumnMapping;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rel.type.RelProtoDataType;
+import org.apache.calcite.sql.SqlCallBinding;
+import org.apache.calcite.sql.SqlOperatorBinding;
 
-import org.eigenbase.rel.metadata.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
 
-import static org.eigenbase.util.Static.RESOURCE;
+import static org.apache.calcite.util.Static.RESOURCE;
 
 /**
  * TableFunctionReturnTypeInference implements rules for deriving table function

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/type/package-info.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/package-info.java b/core/src/main/java/org/apache/calcite/sql/type/package-info.java
index b56b17e..8a396f4 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/package-info.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/package-info.java
@@ -18,6 +18,6 @@
 /**
  * SQL type system.
  */
-package org.eigenbase.sql.type;
+package org.apache.calcite.sql.type;
 
 // End package-info.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/util/ChainedSqlOperatorTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/util/ChainedSqlOperatorTable.java b/core/src/main/java/org/apache/calcite/sql/util/ChainedSqlOperatorTable.java
index 67f2748..538c10a 100644
--- a/core/src/main/java/org/apache/calcite/sql/util/ChainedSqlOperatorTable.java
+++ b/core/src/main/java/org/apache/calcite/sql/util/ChainedSqlOperatorTable.java
@@ -14,11 +14,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.util;
+package org.apache.calcite.sql.util;
 
-import java.util.*;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlOperatorTable;
+import org.apache.calcite.sql.SqlSyntax;
 
-import org.eigenbase.sql.*;
+import java.util.ArrayList;
+import java.util.List;
 
 /**
  * ChainedSqlOperatorTable implements the {@link SqlOperatorTable} interface by

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/util/ListSqlOperatorTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/util/ListSqlOperatorTable.java b/core/src/main/java/org/apache/calcite/sql/util/ListSqlOperatorTable.java
index ee24c32..b018177 100644
--- a/core/src/main/java/org/apache/calcite/sql/util/ListSqlOperatorTable.java
+++ b/core/src/main/java/org/apache/calcite/sql/util/ListSqlOperatorTable.java
@@ -14,11 +14,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.util;
+package org.apache.calcite.sql.util;
 
-import java.util.*;
+import org.apache.calcite.sql.SqlFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlOperatorTable;
+import org.apache.calcite.sql.SqlSyntax;
 
-import org.eigenbase.sql.*;
+import java.util.ArrayList;
+import java.util.List;
 
 /**
  * Implementation of the {@link SqlOperatorTable} interface by using a list of

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 ebed550..97691fa 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
@@ -14,15 +14,29 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.util;
-
-import java.lang.reflect.*;
-import java.util.*;
-
-import org.eigenbase.sql.*;
-import org.eigenbase.util.*;
-
-import com.google.common.collect.*;
+package org.apache.calcite.sql.util;
+
+import org.apache.calcite.sql.SqlBinaryOperator;
+import org.apache.calcite.sql.SqlFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlOperatorTable;
+import org.apache.calcite.sql.SqlPostfixOperator;
+import org.apache.calcite.sql.SqlPrefixOperator;
+import org.apache.calcite.sql.SqlSyntax;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
+
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Multimap;
+
+import java.lang.reflect.Field;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
 
 /**
  * ReflectiveSqlOperatorTable implements the {@link SqlOperatorTable } interface

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/util/SqlBasicVisitor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/util/SqlBasicVisitor.java b/core/src/main/java/org/apache/calcite/sql/util/SqlBasicVisitor.java
index 78518cd..53fd617 100644
--- a/core/src/main/java/org/apache/calcite/sql/util/SqlBasicVisitor.java
+++ b/core/src/main/java/org/apache/calcite/sql/util/SqlBasicVisitor.java
@@ -14,9 +14,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.util;
+package org.apache.calcite.sql.util;
 
-import org.eigenbase.sql.*;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlDataTypeSpec;
+import org.apache.calcite.sql.SqlDynamicParam;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlIntervalQualifier;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlNodeList;
 
 /**
  * Basic implementation of {@link SqlVisitor} which does nothing at each node.
@@ -24,6 +31,8 @@ import org.eigenbase.sql.*;
  * <p>This class is useful as a base class for classes which implement the
  * {@link SqlVisitor} interface. The derived class can override whichever
  * methods it chooses.
+ *
+ * @param <R> Return type
  */
 public class SqlBasicVisitor<R> implements SqlVisitor<R> {
   //~ Methods ----------------------------------------------------------------
@@ -63,13 +72,17 @@ public class SqlBasicVisitor<R> implements SqlVisitor<R> {
 
   //~ Inner Interfaces -------------------------------------------------------
 
-  // REVIEW jvs 16-June-2006:  Without javadoc, the interaction between
-  // ArgHandler and SqlBasicVisitor isn't obvious (nor why this interface
-  // belongs here instead of at top-level).  visitChild already returns
-  // R; why is a separate result() call needed?
+  /** Argument handler. */
   public interface ArgHandler<R> {
+    /** Returns the result of visiting all children of a call to an operator,
+     * then the call itself.
+     *
+     * <p>Typically the result will be the result of the last child visited, or
+     * (if R is {@link Boolean}) whether all children were visited
+     * successfully. */
     R result();
 
+    /** Visits a particular operand of a call, using a given visitor. */
     R visitChild(
         SqlVisitor<R> visitor,
         SqlNode expr,
@@ -80,8 +93,8 @@ public class SqlBasicVisitor<R> implements SqlVisitor<R> {
   //~ Inner Classes ----------------------------------------------------------
 
   /**
-   * Default implementation of {@link ArgHandler} which merely calls {@link
-   * SqlNode#accept} on each operand.
+   * Default implementation of {@link ArgHandler} which merely calls
+   * {@link SqlNode#accept} on each operand.
    */
   public static class ArgHandlerImpl<R> implements ArgHandler<R> {
     private static final ArgHandler INSTANCE = new ArgHandlerImpl();

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/util/SqlBuilder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/util/SqlBuilder.java b/core/src/main/java/org/apache/calcite/sql/util/SqlBuilder.java
index b08fe39..322c31c 100644
--- a/core/src/main/java/org/apache/calcite/sql/util/SqlBuilder.java
+++ b/core/src/main/java/org/apache/calcite/sql/util/SqlBuilder.java
@@ -14,13 +14,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.util;
+package org.apache.calcite.sql.util;
 
-import java.sql.Timestamp;
-import java.util.*;
+import org.apache.calcite.sql.SqlDialect;
+import org.apache.calcite.util.UnmodifiableArrayList;
 
-import org.eigenbase.sql.SqlDialect;
-import org.eigenbase.util.UnmodifiableArrayList;
+import java.sql.Timestamp;
+import java.util.List;
 
 /**
  * Extension to {@link StringBuilder} for the purposes of creating SQL queries
@@ -93,8 +93,7 @@ public class SqlBuilder {
    * @return SQL string
    * @see #getSql()
    */
-  @Override
-  public String toString() {
+  @Override public String toString() {
     return getSql();
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/util/SqlShuttle.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/util/SqlShuttle.java b/core/src/main/java/org/apache/calcite/sql/util/SqlShuttle.java
index dccf411..c530c5c 100644
--- a/core/src/main/java/org/apache/calcite/sql/util/SqlShuttle.java
+++ b/core/src/main/java/org/apache/calcite/sql/util/SqlShuttle.java
@@ -14,11 +14,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.util;
-
-import java.util.*;
-
-import org.eigenbase.sql.*;
+package org.apache.calcite.sql.util;
+
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlDataTypeSpec;
+import org.apache.calcite.sql.SqlDynamicParam;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlIntervalQualifier;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlNodeList;
+
+import java.util.ArrayList;
+import java.util.List;
 
 /**
  * Basic implementation of {@link SqlVisitor} which returns each leaf node

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/util/SqlString.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/util/SqlString.java b/core/src/main/java/org/apache/calcite/sql/util/SqlString.java
index de0a2f1..79a7a48 100644
--- a/core/src/main/java/org/apache/calcite/sql/util/SqlString.java
+++ b/core/src/main/java/org/apache/calcite/sql/util/SqlString.java
@@ -14,9 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.util;
+package org.apache.calcite.sql.util;
 
-import org.eigenbase.sql.SqlDialect;
+import org.apache.calcite.sql.SqlDialect;
 
 /**
  * String that represents a kocher SQL statement, expression, or fragment.
@@ -43,13 +43,11 @@ public class SqlString {
     assert dialect != null;
   }
 
-  @Override
-  public int hashCode() {
+  @Override public int hashCode() {
     return s.hashCode();
   }
 
-  @Override
-  public boolean equals(Object obj) {
+  @Override public boolean equals(Object obj) {
     return obj instanceof SqlString
         && s.equals(((SqlString) obj).s);
   }
@@ -62,8 +60,7 @@ public class SqlString {
    * @return SQL string
    * @see #getSql()
    */
-  @Override
-  public String toString() {
+  @Override public String toString() {
     return s;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/util/SqlVisitor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/util/SqlVisitor.java b/core/src/main/java/org/apache/calcite/sql/util/SqlVisitor.java
index be1bb79..ff45de8 100644
--- a/core/src/main/java/org/apache/calcite/sql/util/SqlVisitor.java
+++ b/core/src/main/java/org/apache/calcite/sql/util/SqlVisitor.java
@@ -14,13 +14,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.util;
+package org.apache.calcite.sql.util;
 
-import org.eigenbase.sql.*;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlDataTypeSpec;
+import org.apache.calcite.sql.SqlDynamicParam;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlIntervalQualifier;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlNodeList;
+import org.apache.calcite.sql.SqlOperator;
 
 /**
- * Visitor class, follows the {@link org.eigenbase.util.Glossary#VISITOR_PATTERN
- * visitor pattern}.
+ * Visitor class, follows the
+ * {@link org.apache.calcite.util.Glossary#VISITOR_PATTERN visitor pattern}.
  *
  * <p>The type parameter <code>R</code> is the return type of each <code>
  * visit()</code> method. If the methods do not need to return a value, use
@@ -29,6 +37,8 @@ import org.eigenbase.sql.*;
  * @see SqlBasicVisitor
  * @see SqlNode#accept(SqlVisitor)
  * @see SqlOperator#acceptCall
+ *
+ * @param <R> Return type
  */
 public interface SqlVisitor<R> {
   //~ Methods ----------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/util/package-info.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/util/package-info.java b/core/src/main/java/org/apache/calcite/sql/util/package-info.java
index 3d957bf..22f73b5 100644
--- a/core/src/main/java/org/apache/calcite/sql/util/package-info.java
+++ b/core/src/main/java/org/apache/calcite/sql/util/package-info.java
@@ -18,6 +18,6 @@
 /**
  * Utility classes for the SQL object model, parsing, and validation.
  */
-package org.eigenbase.sql.util;
+package org.apache.calcite.sql.util;
 
 // End package-info.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 d46d792..9dc0178 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
@@ -14,16 +14,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.validate;
+package org.apache.calcite.sql.validate;
 
-import java.util.*;
-
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.util.*;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
 
+import java.util.List;
+
 /**
  * Abstract implementation of {@link SqlValidatorNamespace}.
  */
@@ -110,8 +112,8 @@ abstract class AbstractNamespace implements SqlValidatorNamespace {
 
   /**
    * Validates this scope and returns the type of the records it returns.
-   * External users should call {@link #validate}, which uses the {@link
-   * #status} field to protect against cycles.
+   * External users should call {@link #validate}, which uses the
+   * {@link #status} field to protect against cycles.
    *
    * @return record data type, never null
    */

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/validate/AggChecker.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/AggChecker.java b/core/src/main/java/org/apache/calcite/sql/validate/AggChecker.java
index 0c7867a..1b7d3a8 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/AggChecker.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/AggChecker.java
@@ -14,14 +14,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.validate;
+package org.apache.calcite.sql.validate;
 
-import java.util.*;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlUtil;
+import org.apache.calcite.sql.util.SqlBasicVisitor;
 
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.util.*;
+import java.util.List;
+import java.util.Stack;
 
-import static org.eigenbase.util.Static.RESOURCE;
+import static org.apache.calcite.util.Static.RESOURCE;
 
 /**
  * Visitor which throws an exception if any component of the expression is not a

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/validate/AggFinder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/AggFinder.java b/core/src/main/java/org/apache/calcite/sql/validate/AggFinder.java
index 1ac5e41..4df2132 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/AggFinder.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/AggFinder.java
@@ -14,19 +14,26 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.validate;
+package org.apache.calcite.sql.validate;
 
-import java.util.List;
-
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.util.*;
-import org.eigenbase.util.*;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlOperatorTable;
+import org.apache.calcite.sql.SqlSyntax;
+import org.apache.calcite.sql.util.SqlBasicVisitor;
+import org.apache.calcite.util.Util;
 
 import com.google.common.collect.Lists;
 
+import java.util.List;
+
 /**
- * Visitor which looks for an aggregate function inside a tree of {@link
- * SqlNode} objects.
+ * Visitor which looks for an aggregate function inside a tree of
+ * {@link SqlNode} objects.
  */
 class AggFinder extends SqlBasicVisitor<Void> {
   //~ Instance fields --------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/validate/AggregatingScope.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/AggregatingScope.java b/core/src/main/java/org/apache/calcite/sql/validate/AggregatingScope.java
index ca0431e..aadb622 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/AggregatingScope.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/AggregatingScope.java
@@ -14,9 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.validate;
+package org.apache.calcite.sql.validate;
 
-import org.eigenbase.sql.*;
+import org.apache.calcite.sql.SqlNode;
 
 /**
  * An extension to the {@link SqlValidatorScope} interface which indicates that

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/validate/AggregatingSelectScope.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/AggregatingSelectScope.java b/core/src/main/java/org/apache/calcite/sql/validate/AggregatingSelectScope.java
index 6a8614f..27875ba 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/AggregatingSelectScope.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/AggregatingSelectScope.java
@@ -14,14 +14,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.validate;
+package org.apache.calcite.sql.validate;
 
-import java.util.*;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlNodeList;
+import org.apache.calcite.sql.SqlSelect;
 
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.fun.*;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
 
-import static org.eigenbase.sql.SqlUtil.stripAs;
+import static org.apache.calcite.sql.SqlUtil.stripAs;
 
 /**
  * Scope for resolving identifiers within a SELECT statement that has a

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/validate/AliasNamespace.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/AliasNamespace.java b/core/src/main/java/org/apache/calcite/sql/validate/AliasNamespace.java
index b216823..c109dcf 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/AliasNamespace.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/AliasNamespace.java
@@ -14,16 +14,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.validate;
+package org.apache.calcite.sql.validate;
 
-import java.util.*;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.util.Util;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.fun.*;
-import org.eigenbase.util.Util;
+import java.util.ArrayList;
+import java.util.List;
 
-import static org.eigenbase.util.Static.RESOURCE;
+import static org.apache.calcite.util.Static.RESOURCE;
 
 /**
  * Namespace for an <code>AS t(c1, c2, ...)</code> clause.
@@ -115,8 +119,7 @@ public class AliasNamespace extends AbstractNamespace {
       }
       ++i;
     }
-    throw new AssertionError(
-        "unknown field '" + name
+    throw new AssertionError("unknown field '" + name
         + "' in rowtype " + underlyingRowType);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/validate/CollectNamespace.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/CollectNamespace.java b/core/src/main/java/org/apache/calcite/sql/validate/CollectNamespace.java
index ba45c9c..f076358 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/CollectNamespace.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/CollectNamespace.java
@@ -14,10 +14,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.validate;
+package org.apache.calcite.sql.validate;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
+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;
 
 /**
  * Namespace for COLLECT and TABLE constructs.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/validate/CollectScope.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/CollectScope.java b/core/src/main/java/org/apache/calcite/sql/validate/CollectScope.java
index 65031c7..b14251d 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/CollectScope.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/CollectScope.java
@@ -14,9 +14,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.validate;
+package org.apache.calcite.sql.validate;
 
-import org.eigenbase.sql.*;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlNode;
 
 /**
  * The name-resolution context for expression inside a multiset call. The

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/validate/DelegatingNamespace.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/DelegatingNamespace.java b/core/src/main/java/org/apache/calcite/sql/validate/DelegatingNamespace.java
index dd216ba..6dfb3fd 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/DelegatingNamespace.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/DelegatingNamespace.java
@@ -14,13 +14,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.validate;
+package org.apache.calcite.sql.validate;
 
-import java.util.*;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.util.Pair;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.util.*;
+import java.util.List;
 
 /**
  * An implementation of {@link SqlValidatorNamespace} that delegates all methods

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/validate/DelegatingScope.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/DelegatingScope.java b/core/src/main/java/org/apache/calcite/sql/validate/DelegatingScope.java
index 2ab2a57..66deffb 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/DelegatingScope.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/DelegatingScope.java
@@ -14,17 +14,23 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.validate;
-
-import java.util.*;
-
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.parser.*;
+package org.apache.calcite.sql.validate;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlNodeList;
+import org.apache.calcite.sql.SqlSelect;
+import org.apache.calcite.sql.SqlWindow;
+import org.apache.calcite.sql.parser.SqlParserPos;
 
 import com.google.common.collect.ImmutableList;
 
-import static org.eigenbase.util.Static.RESOURCE;
+import java.util.List;
+
+import static org.apache.calcite.util.Static.RESOURCE;
 
 /**
  * A scope which delegates all requests to its parent scope. Use this as a base
@@ -37,8 +43,8 @@ public abstract class DelegatingScope implements SqlValidatorScope {
    * Parent scope. This is where to look next to resolve an identifier; it is
    * not always the parent object in the parse tree.
    *
-   * <p>This is never null: at the top of the tree, it is an {@link
-   * EmptyScope}.
+   * <p>This is never null: at the top of the tree, it is an
+   * {@link EmptyScope}.
    */
   protected final SqlValidatorScope parent;
   protected final SqlValidatorImpl validator;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/validate/DelegatingSqlValidatorCatalogReader.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/DelegatingSqlValidatorCatalogReader.java b/core/src/main/java/org/apache/calcite/sql/validate/DelegatingSqlValidatorCatalogReader.java
index 11b0e85..b1d9c39 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/DelegatingSqlValidatorCatalogReader.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/DelegatingSqlValidatorCatalogReader.java
@@ -14,16 +14,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.validate;
+package org.apache.calcite.sql.validate;
 
-import java.util.List;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlIdentifier;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
+import java.util.List;
 
 /**
  * Implementation of
- * {@link org.eigenbase.sql.validate.SqlValidatorCatalogReader} that passes
+ * {@link org.apache.calcite.sql.validate.SqlValidatorCatalogReader} that passes
  * all calls to a parent catalog reader.
  */
 public abstract class DelegatingSqlValidatorCatalogReader

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/validate/DelegatingSqlValidatorTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/DelegatingSqlValidatorTable.java b/core/src/main/java/org/apache/calcite/sql/validate/DelegatingSqlValidatorTable.java
index b764cef..2a3e352 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/DelegatingSqlValidatorTable.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/DelegatingSqlValidatorTable.java
@@ -14,15 +14,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.validate;
+package org.apache.calcite.sql.validate;
 
-import java.util.List;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlAccessType;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
+import java.util.List;
 
 /**
- * Implements {@link org.eigenbase.sql.validate.SqlValidatorTable} by
+ * Implements {@link org.apache.calcite.sql.validate.SqlValidatorTable} by
  * delegating to a parent table.
  */
 public abstract class DelegatingSqlValidatorTable implements SqlValidatorTable {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/validate/EmptyScope.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/EmptyScope.java b/core/src/main/java/org/apache/calcite/sql/validate/EmptyScope.java
index d948172..db4fefd 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/EmptyScope.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/EmptyScope.java
@@ -14,14 +14,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.validate;
+package org.apache.calcite.sql.validate;
 
-import java.util.*;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlDataTypeSpec;
+import org.apache.calcite.sql.SqlDynamicParam;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlNodeList;
+import org.apache.calcite.sql.SqlWindow;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
+import java.util.List;
 
-import static org.eigenbase.util.Static.RESOURCE;
+import static org.apache.calcite.util.Static.RESOURCE;
 
 /**
  * Deviant implementation of {@link SqlValidatorScope} for the top of the scope

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/validate/FieldNamespace.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/FieldNamespace.java b/core/src/main/java/org/apache/calcite/sql/validate/FieldNamespace.java
index 4af678a..bf6fcbe 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/FieldNamespace.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/FieldNamespace.java
@@ -14,10 +14,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.validate;
+package org.apache.calcite.sql.validate;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlNode;
 
 /**
  * Implementation of {@link SqlValidatorNamespace} for a field of a record.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/validate/IdentifierNamespace.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/IdentifierNamespace.java b/core/src/main/java/org/apache/calcite/sql/validate/IdentifierNamespace.java
index 9a9ead2..8e08504 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/IdentifierNamespace.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/IdentifierNamespace.java
@@ -14,22 +14,26 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.validate;
+package org.apache.calcite.sql.validate;
 
-import java.util.*;
-
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.util.*;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.util.Pair;
 
 import com.google.common.collect.ImmutableList;
 
-import static org.eigenbase.util.Static.RESOURCE;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import static org.apache.calcite.util.Static.RESOURCE;
 
 /**
- * Namespace whose contents are defined by the type of an {@link
- * org.eigenbase.sql.SqlIdentifier identifier}.
+ * Namespace whose contents are defined by the type of an
+ * {@link org.apache.calcite.sql.SqlIdentifier identifier}.
  */
 public class IdentifierNamespace extends AbstractNamespace {
   //~ Instance fields --------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/validate/JoinNamespace.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/JoinNamespace.java b/core/src/main/java/org/apache/calcite/sql/validate/JoinNamespace.java
index 62d39b2..464d84c 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/JoinNamespace.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/JoinNamespace.java
@@ -14,10 +14,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.validate;
+package org.apache.calcite.sql.validate;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.sql.SqlJoin;
+import org.apache.calcite.sql.SqlNode;
 
 /**
  * Namespace representing the row type produced by joining two relations.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/validate/JoinScope.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/JoinScope.java b/core/src/main/java/org/apache/calcite/sql/validate/JoinScope.java
index 56ed234..e8272d5 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/JoinScope.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/JoinScope.java
@@ -14,9 +14,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.validate;
+package org.apache.calcite.sql.validate;
 
-import org.eigenbase.sql.*;
+import org.apache.calcite.sql.SqlJoin;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlWindow;
 
 /**
  * The name-resolution context for expression inside a JOIN clause. The objects

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/validate/ListScope.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/ListScope.java b/core/src/main/java/org/apache/calcite/sql/validate/ListScope.java
index 2e88854..80af917 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/ListScope.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/ListScope.java
@@ -14,15 +14,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.validate;
+package org.apache.calcite.sql.validate;
 
-import java.util.*;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.util.*;
+import java.util.ArrayList;
+import java.util.List;
 
-import static org.eigenbase.util.Static.RESOURCE;
+import static org.apache.calcite.util.Static.RESOURCE;
 
 /**
  * Abstract base for a scope which is defined by a list of child namespaces and

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/validate/OrderByScope.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/OrderByScope.java b/core/src/main/java/org/apache/calcite/sql/validate/OrderByScope.java
index dbe997b..ea38156 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/OrderByScope.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/OrderByScope.java
@@ -14,12 +14,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.validate;
+package org.apache.calcite.sql.validate;
 
-import java.util.*;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlNodeList;
+import org.apache.calcite.sql.SqlSelect;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
+import java.util.List;
 
 /**
  * Represents the name-resolution context for expressions in an ORDER BY clause.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/validate/OverScope.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/OverScope.java b/core/src/main/java/org/apache/calcite/sql/validate/OverScope.java
index 307d576..7f980e2 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/OverScope.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/OverScope.java
@@ -14,12 +14,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.validate;
+package org.apache.calcite.sql.validate;
 
-import java.util.*;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.util.Pair;
 
-import org.eigenbase.sql.*;
-import org.eigenbase.util.*;
+import java.util.List;
 
 /**
  * The name-resolution scope of a OVER clause. The objects visible are those in

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/validate/ParameterNamespace.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/ParameterNamespace.java b/core/src/main/java/org/apache/calcite/sql/validate/ParameterNamespace.java
index e064ef2..9d9d8ba 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/ParameterNamespace.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/ParameterNamespace.java
@@ -14,10 +14,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.validate;
+package org.apache.calcite.sql.validate;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlNode;
 
 /**
  * Namespace representing the type of a dynamic parameter.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/validate/ParameterScope.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/ParameterScope.java b/core/src/main/java/org/apache/calcite/sql/validate/ParameterScope.java
index f96b8dd..6a5d86c 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/ParameterScope.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/ParameterScope.java
@@ -14,16 +14,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.validate;
+package org.apache.calcite.sql.validate;
 
-import java.util.*;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlIdentifier;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
+import java.util.Map;
 
 /**
- * A scope which contains nothing besides a few parameters. Like {@link
- * EmptyScope} (which is its base class), it has no parent scope.
+ * A scope which contains nothing besides a few parameters. Like
+ * {@link EmptyScope} (which is its base class), it has no parent scope.
  *
  * @see ParameterNamespace
  */

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/validate/ProcedureNamespace.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/ProcedureNamespace.java b/core/src/main/java/org/apache/calcite/sql/validate/ProcedureNamespace.java
index 535e8a1..9755877 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/ProcedureNamespace.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/ProcedureNamespace.java
@@ -14,11 +14,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.validate;
+package org.apache.calcite.sql.validate;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.type.SqlTypeName;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.type.SqlTypeName;
 
 /**
  * Namespace whose contents are defined by the result of a call to a

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/validate/SelectNamespace.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SelectNamespace.java b/core/src/main/java/org/apache/calcite/sql/validate/SelectNamespace.java
index 3d289bc..1ba7347 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SelectNamespace.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SelectNamespace.java
@@ -14,11 +14,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.validate;
+package org.apache.calcite.sql.validate;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.type.*;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlSelect;
+import org.apache.calcite.sql.type.SqlTypeUtil;
 
 /**
  * Namespace offered by a subquery.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/validate/SelectScope.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SelectScope.java b/core/src/main/java/org/apache/calcite/sql/validate/SelectScope.java
index cc4e594..76ddeb1 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SelectScope.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SelectScope.java
@@ -14,22 +14,28 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.validate;
+package org.apache.calcite.sql.validate;
 
-import java.util.*;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlNodeList;
+import org.apache.calcite.sql.SqlSelect;
+import org.apache.calcite.sql.SqlWindow;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.util.Pair;
 
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.fun.*;
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.util.*;
+import java.util.ArrayList;
+import java.util.List;
 
 /**
  * The name-resolution scope of a SELECT clause. The objects visible are those
  * in the FROM clause, and objects inherited from the parent scope.
  *
  *
- * <p>This object is both a {@link SqlValidatorScope} and a {@link
- * SqlValidatorNamespace}. In the query</p>
+ * <p>This object is both a {@link SqlValidatorScope} and a
+ * {@link SqlValidatorNamespace}. In the query</p>
  *
  * <blockquote>
  * <pre>SELECT name FROM (

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 5432080..ac71d01 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
@@ -14,13 +14,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.validate;
+package org.apache.calcite.sql.validate;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.util.*;
+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.eigenbase.util.Static.RESOURCE;
+import static org.apache.calcite.util.Static.RESOURCE;
 
 /**
  * Namespace based upon a set operation (UNION, INTERSECT, EXCEPT).

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/validate/SqlConformance.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlConformance.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlConformance.java
index 1d3310e..27bed63 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlConformance.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlConformance.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.validate;
+package org.apache.calcite.sql.validate;
 
 /**
  * Enumeration of valid SQL compatiblity modes.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 4fa5658..e4f498b 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
@@ -14,12 +14,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.validate;
+package org.apache.calcite.sql.validate;
 
-import java.util.List;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.util.Util;
 
-import org.eigenbase.sql.*;
-import org.eigenbase.util.*;
+import java.util.List;
 
 /**
  * An implementation of {@link SqlMoniker} that encapsulates the normalized name

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/validate/SqlMoniker.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlMoniker.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlMoniker.java
index 7be546b..72f8ffd 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlMoniker.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlMoniker.java
@@ -14,11 +14,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.validate;
+package org.apache.calcite.sql.validate;
 
-import java.util.List;
+import org.apache.calcite.sql.SqlIdentifier;
 
-import org.eigenbase.sql.*;
+import java.util.List;
 
 /**
  * An interface of an object identifier that represents a SqlIdentifier

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/validate/SqlMonikerComparator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlMonikerComparator.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlMonikerComparator.java
index 89e3daf..85eb1c0 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlMonikerComparator.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlMonikerComparator.java
@@ -14,13 +14,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql.validate;
+package org.apache.calcite.sql.validate;
 
-import java.util.*;
+import java.util.Comparator;
 
 /**
- * A general-purpose implementation of {@link Comparator} to compare {@link
- * SqlMoniker} values.
+ * A general-purpose implementation of {@link Comparator} to compare
+ * {@link SqlMoniker} values.
  */
 public class SqlMonikerComparator implements Comparator<SqlMoniker> {
   //~ Methods ----------------------------------------------------------------


[15/58] [abbrv] [partial] incubator-calcite git commit: [CALCITE-306] Standardize code style for "import package.*; "

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/test/JdbcTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/JdbcTest.java b/core/src/test/java/org/apache/calcite/test/JdbcTest.java
index 07a80a8..291dff1 100644
--- a/core/src/test/java/org/apache/calcite/test/JdbcTest.java
+++ b/core/src/test/java/org/apache/calcite/test/JdbcTest.java
@@ -14,69 +14,131 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.test;
-
-import net.hydromatic.avatica.*;
-
-import net.hydromatic.linq4j.*;
-import net.hydromatic.linq4j.expressions.Types;
-import net.hydromatic.linq4j.function.Function1;
-import net.hydromatic.linq4j.function.Function2;
-
-import net.hydromatic.optiq.*;
-import net.hydromatic.optiq.impl.*;
-import net.hydromatic.optiq.impl.clone.CloneSchema;
-import net.hydromatic.optiq.impl.generate.RangeTable;
-import net.hydromatic.optiq.impl.java.AbstractQueryableTable;
-import net.hydromatic.optiq.impl.java.JavaTypeFactory;
-import net.hydromatic.optiq.impl.java.ReflectiveSchema;
-import net.hydromatic.optiq.impl.jdbc.JdbcConvention;
-import net.hydromatic.optiq.impl.jdbc.JdbcSchema;
-import net.hydromatic.optiq.jdbc.*;
-import net.hydromatic.optiq.jdbc.Driver;
-import net.hydromatic.optiq.prepare.OptiqPrepareImpl;
-import net.hydromatic.optiq.prepare.Prepare;
-import net.hydromatic.optiq.runtime.Hook;
-import net.hydromatic.optiq.runtime.SqlFunctions;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.reltype.RelDataTypeFactory;
-import org.eigenbase.reltype.RelProtoDataType;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.advise.SqlAdvisorGetHintsFunction;
-import org.eigenbase.sql.parser.SqlParserUtil;
-import org.eigenbase.sql.type.SqlTypeName;
-import org.eigenbase.test.DiffTestCase;
-import org.eigenbase.util.Bug;
-import org.eigenbase.util.Pair;
-import org.eigenbase.util.Util;
+package org.apache.calcite.test;
+
+import org.apache.calcite.adapter.clone.CloneSchema;
+import org.apache.calcite.adapter.generate.RangeTable;
+import org.apache.calcite.adapter.java.AbstractQueryableTable;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.adapter.java.ReflectiveSchema;
+import org.apache.calcite.adapter.jdbc.JdbcConvention;
+import org.apache.calcite.adapter.jdbc.JdbcSchema;
+import org.apache.calcite.avatica.AvaticaConnection;
+import org.apache.calcite.avatica.AvaticaStatement;
+import org.apache.calcite.avatica.Handler;
+import org.apache.calcite.avatica.HandlerImpl;
+import org.apache.calcite.avatica.Meta;
+import org.apache.calcite.jdbc.CalciteConnection;
+import org.apache.calcite.jdbc.CalciteSchema;
+import org.apache.calcite.jdbc.Driver;
+import org.apache.calcite.jdbc.MetaImpl;
+import org.apache.calcite.linq4j.BaseQueryable;
+import org.apache.calcite.linq4j.Enumerable;
+import org.apache.calcite.linq4j.Enumerator;
+import org.apache.calcite.linq4j.Linq4j;
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.linq4j.QueryProvider;
+import org.apache.calcite.linq4j.Queryable;
+import org.apache.calcite.linq4j.function.Function1;
+import org.apache.calcite.linq4j.function.Function2;
+import org.apache.calcite.linq4j.tree.Types;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.prepare.CalcitePrepareImpl;
+import org.apache.calcite.prepare.Prepare;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.TableModify;
+import org.apache.calcite.rel.logical.LogicalTableModify;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelProtoDataType;
+import org.apache.calcite.runtime.Hook;
+import org.apache.calcite.runtime.SqlFunctions;
+import org.apache.calcite.schema.ModifiableTable;
+import org.apache.calcite.schema.QueryableTable;
+import org.apache.calcite.schema.Schema;
+import org.apache.calcite.schema.SchemaFactory;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.schema.Table;
+import org.apache.calcite.schema.TableFactory;
+import org.apache.calcite.schema.TableFunction;
+import org.apache.calcite.schema.TableMacro;
+import org.apache.calcite.schema.TranslatableTable;
+import org.apache.calcite.schema.impl.AbstractSchema;
+import org.apache.calcite.schema.impl.AbstractTable;
+import org.apache.calcite.schema.impl.AbstractTableQueryable;
+import org.apache.calcite.schema.impl.TableFunctionImpl;
+import org.apache.calcite.schema.impl.TableMacroImpl;
+import org.apache.calcite.schema.impl.ViewTable;
+import org.apache.calcite.sql.SqlDialect;
+import org.apache.calcite.sql.SqlSelect;
+import org.apache.calcite.sql.advise.SqlAdvisorGetHintsFunction;
+import org.apache.calcite.sql.parser.SqlParserUtil;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.util.Bug;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
 
 import com.google.common.base.Function;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableMultimap;
 
+import net.hydromatic.quidem.Quidem;
+
 import org.hsqldb.jdbcDriver;
 
 import org.junit.Ignore;
 import org.junit.Test;
 
-import java.io.*;
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.FileWriter;
+import java.io.IOException;
 import java.lang.reflect.Method;
 import java.math.BigDecimal;
 import java.net.URL;
-import java.sql.*;
+import java.sql.Array;
+import java.sql.Connection;
+import java.sql.DatabaseMetaData;
 import java.sql.Date;
+import java.sql.DriverManager;
+import java.sql.DriverPropertyInfo;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
 import java.sql.Statement;
-import java.util.*;
+import java.sql.Timestamp;
+import java.util.AbstractList;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Calendar;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.TimeZone;
 import javax.sql.DataSource;
 
-import net.hydromatic.quidem.Quidem;
-
-import static org.hamcrest.CoreMatchers.*;
-import static org.junit.Assert.*;
+import static org.hamcrest.CoreMatchers.containsString;
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.CoreMatchers.instanceOf;
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.CoreMatchers.not;
+import static org.hamcrest.CoreMatchers.notNullValue;
+import static org.hamcrest.CoreMatchers.nullValue;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 /**
  * Tests for using Calcite via JDBC.
@@ -104,20 +166,18 @@ public class JdbcTest {
       Types.lookupMethod(JdbcTest.class, "processCursors",
           int.class, Enumerable.class, Enumerable.class);
 
-  public static final String FOODMART_SCHEMA =
-      "     {\n"
+  public static final String FOODMART_SCHEMA = "     {\n"
       + "       type: 'jdbc',\n"
       + "       name: 'foodmart',\n"
-      + "       jdbcDriver: '" + OptiqAssert.CONNECTION_SPEC.driver + "',\n"
-      + "       jdbcUser: '" + OptiqAssert.CONNECTION_SPEC.username + "',\n"
-      + "       jdbcPassword: '" + OptiqAssert.CONNECTION_SPEC.password + "',\n"
-      + "       jdbcUrl: '" + OptiqAssert.CONNECTION_SPEC.url + "',\n"
+      + "       jdbcDriver: '" + CalciteAssert.CONNECTION_SPEC.driver + "',\n"
+      + "       jdbcUser: '" + CalciteAssert.CONNECTION_SPEC.username + "',\n"
+      + "       jdbcPassword: '" + CalciteAssert.CONNECTION_SPEC.password + "',\n"
+      + "       jdbcUrl: '" + CalciteAssert.CONNECTION_SPEC.url + "',\n"
       + "       jdbcCatalog: null,\n"
       + "       jdbcSchema: 'foodmart'\n"
       + "     }\n";
 
-  public static final String FOODMART_MODEL =
-      "{\n"
+  public static final String FOODMART_MODEL = "{\n"
       + "  version: '1.0',\n"
       + "  defaultSchema: 'foodmart',\n"
       + "   schemas: [\n"
@@ -125,8 +185,7 @@ public class JdbcTest {
       + "   ]\n"
       + "}";
 
-  public static final String HR_SCHEMA =
-      "     {\n"
+  public static final String HR_SCHEMA = "     {\n"
       + "       type: 'custom',\n"
       + "       name: 'hr',\n"
       + "       factory: '"
@@ -137,8 +196,7 @@ public class JdbcTest {
       + "       }\n"
       + "     }\n";
 
-  public static final String HR_MODEL =
-      "{\n"
+  public static final String HR_MODEL = "{\n"
       + "  version: '1.0',\n"
       + "  defaultSchema: 'hr',\n"
       + "   schemas: [\n"
@@ -146,9 +204,7 @@ public class JdbcTest {
       + "   ]\n"
       + "}";
 
-
-  public static final String START_OF_GROUP_DATA =
-      "(values"
+  public static final String START_OF_GROUP_DATA = "(values"
       + "(1,0,1),\n"
       + "(2,0,1),\n"
       + "(3,1,2),\n"
@@ -168,21 +224,19 @@ public class JdbcTest {
    */
   @Test public void testTableFunction()
       throws SQLException, ClassNotFoundException {
-    Class.forName("net.hydromatic.optiq.jdbc.Driver");
     Connection connection =
         DriverManager.getConnection("jdbc:calcite:");
-    OptiqConnection optiqConnection =
-        connection.unwrap(OptiqConnection.class);
-    SchemaPlus rootSchema = optiqConnection.getRootSchema();
+    CalciteConnection calciteConnection =
+        connection.unwrap(CalciteConnection.class);
+    SchemaPlus rootSchema = calciteConnection.getRootSchema();
     SchemaPlus schema = rootSchema.add("s", new AbstractSchema());
     final TableFunction table =
         TableFunctionImpl.create(GENERATE_STRINGS_METHOD);
     schema.add("GenerateStrings", table);
-    ResultSet resultSet = connection.createStatement().executeQuery(
-        "select *\n"
+    ResultSet resultSet = connection.createStatement().executeQuery("select *\n"
         + "from table(\"s\".\"GenerateStrings\"(5)) as t(n, c)\n"
         + "where char_length(c) > 3");
-    assertThat(OptiqAssert.toString(resultSet),
+    assertThat(CalciteAssert.toString(resultSet),
         equalTo("N=4; C=abcd\n"));
   }
 
@@ -193,14 +247,12 @@ public class JdbcTest {
   @Test public void testTableFunctionDynamicStructure()
       throws SQLException, ClassNotFoundException {
     Connection connection = getConnectionWithMultiplyFunction();
-    final PreparedStatement ps = connection.prepareStatement(
-        "select *\n"
+    final PreparedStatement ps = connection.prepareStatement("select *\n"
         + "from table(\"s\".\"multiplication\"(4, 3, ?))\n");
     ps.setInt(1, 100);
     ResultSet resultSet = ps.executeQuery();
-    assertThat(OptiqAssert.toString(resultSet),
-        equalTo(
-            "row_name=row 0; c1=101; c2=102; c3=103; c4=104\n"
+    assertThat(CalciteAssert.toString(resultSet),
+        equalTo("row_name=row 0; c1=101; c2=102; c3=103; c4=104\n"
             + "row_name=row 1; c1=102; c2=104; c3=106; c4=108\n"
             + "row_name=row 2; c1=103; c2=106; c3=109; c4=112\n"));
   }
@@ -214,18 +266,16 @@ public class JdbcTest {
       throws SQLException, ClassNotFoundException {
     Connection connection = getConnectionWithMultiplyFunction();
     try {
-      final PreparedStatement ps = connection.prepareStatement(
-          "select *\n"
+      final PreparedStatement ps = connection.prepareStatement("select *\n"
           + "from table(\"s\".\"multiplication\"(?, 3, 100))\n");
       ps.setInt(1, 100);
       ResultSet resultSet = ps.executeQuery();
       fail("Should fail, got " + resultSet);
     } catch (SQLException e) {
       assertThat(e.getMessage(),
-          containsString(
-              "Wrong arguments for table function 'public static "
-              + "net.hydromatic.optiq.QueryableTable net"
-              + ".hydromatic.optiq.test.JdbcTest"
+          containsString("Wrong arguments for table function 'public static "
+              + "org.apache.calcite.schema.QueryableTable "
+              + "org.apache.calcite.test.JdbcTest"
               + ".multiplicationTable(int,int,java.lang.Integer)'"
               + " call. Expected '[int, int, class"
               + "java.lang.Integer]', actual '[null, 3, 100]'"));
@@ -234,12 +284,11 @@ public class JdbcTest {
 
   private Connection getConnectionWithMultiplyFunction()
       throws ClassNotFoundException, SQLException {
-    Class.forName("net.hydromatic.optiq.jdbc.Driver");
     Connection connection =
         DriverManager.getConnection("jdbc:calcite:");
-    OptiqConnection optiqConnection =
-        connection.unwrap(OptiqConnection.class);
-    SchemaPlus rootSchema = optiqConnection.getRootSchema();
+    CalciteConnection calciteConnection =
+        connection.unwrap(CalciteConnection.class);
+    SchemaPlus rootSchema = calciteConnection.getRootSchema();
     SchemaPlus schema = rootSchema.add("s", new AbstractSchema());
     final TableFunction table =
         TableFunctionImpl.create(MULTIPLICATION_TABLE_METHOD);
@@ -254,12 +303,11 @@ public class JdbcTest {
           + "could not be implemented")
   @Test public void testTableFunctionCursorInputs()
       throws SQLException, ClassNotFoundException {
-    Class.forName("net.hydromatic.optiq.jdbc.Driver");
     Connection connection =
         DriverManager.getConnection("jdbc:calcite:");
-    OptiqConnection optiqConnection =
-        connection.unwrap(OptiqConnection.class);
-    SchemaPlus rootSchema = optiqConnection.getRootSchema();
+    CalciteConnection calciteConnection =
+        connection.unwrap(CalciteConnection.class);
+    SchemaPlus rootSchema = calciteConnection.getRootSchema();
     SchemaPlus schema = rootSchema.add("s", new AbstractSchema());
     final TableFunction table =
         TableFunctionImpl.create(GENERATE_STRINGS_METHOD);
@@ -267,8 +315,7 @@ public class JdbcTest {
     final TableFunction add =
         TableFunctionImpl.create(PROCESS_CURSOR_METHOD);
     schema.add("process", add);
-    final PreparedStatement ps = connection.prepareStatement(
-        "select *\n"
+    final PreparedStatement ps = connection.prepareStatement("select *\n"
         + "from table(\"s\".\"process\"(2,\n"
         + "cursor(select * from table(\"s\".\"GenerateStrings\"(?)))\n"
         + ")) as t(u)\n"
@@ -277,9 +324,8 @@ public class JdbcTest {
     ResultSet resultSet = ps.executeQuery();
     // GenerateStrings returns 0..4, then 2 is added (process function),
     // thus 2..6, finally where u > 3 leaves just 4..6
-    assertThat(OptiqAssert.toString(resultSet),
-        equalTo(
-            "u=4\n"
+    assertThat(CalciteAssert.toString(resultSet),
+        equalTo("u=4\n"
             + "u=5\n"
             + "u=6\n"));
   }
@@ -291,12 +337,11 @@ public class JdbcTest {
           + "could not be implemented")
   @Test public void testTableFunctionCursorsInputs()
       throws SQLException, ClassNotFoundException {
-    Class.forName("net.hydromatic.optiq.jdbc.Driver");
     Connection connection =
         getConnectionWithMultiplyFunction();
-    OptiqConnection optiqConnection =
-        connection.unwrap(OptiqConnection.class);
-    SchemaPlus rootSchema = optiqConnection.getRootSchema();
+    CalciteConnection calciteConnection =
+        connection.unwrap(CalciteConnection.class);
+    SchemaPlus rootSchema = calciteConnection.getRootSchema();
     SchemaPlus schema = rootSchema.getSubSchema("s");
     final TableFunction table =
         TableFunctionImpl.create(GENERATE_STRINGS_METHOD);
@@ -304,8 +349,7 @@ public class JdbcTest {
     final TableFunction add =
         TableFunctionImpl.create(PROCESS_CURSORS_METHOD);
     schema.add("process", add);
-    final PreparedStatement ps = connection.prepareStatement(
-        "select *\n"
+    final PreparedStatement ps = connection.prepareStatement("select *\n"
         + "from table(\"s\".\"process\"(2,\n"
         + "cursor(select * from table(\"s\".\"multiplication\"(5,5,0))),\n"
         + "cursor(select * from table(\"s\".\"GenerateStrings\"(?)))\n"
@@ -317,9 +361,8 @@ public class JdbcTest {
     // multiplication produce 1..5
     // process sums and adds 2
     // sum is 2 + 1..9 == 3..9
-    assertThat(OptiqAssert.toString(resultSet),
-        equalTo(
-            "u=4\n"
+    assertThat(CalciteAssert.toString(resultSet),
+        equalTo("u=4\n"
             + "u=5\n"
             + "u=6\n"
             + "u=7\n"
@@ -328,25 +371,23 @@ public class JdbcTest {
   }
 
   /**
-   * Tests {@link org.eigenbase.sql.advise.SqlAdvisorGetHintsFunction}.
+   * Tests {@link org.apache.calcite.sql.advise.SqlAdvisorGetHintsFunction}.
    */
   @Test public void testSqlAdvisorGetHintsFunction()
       throws SQLException, ClassNotFoundException {
     String res = adviseSql("select e.e^ from \"emps\" e");
     assertThat(res,
-        equalTo(
-            "id=e; names=null; type=MATCH\n"
+        equalTo("id=e; names=null; type=MATCH\n"
             + "id=empid; names=[empid]; type=COLUMN\n"));
   }
   /**
-   * Tests {@link org.eigenbase.sql.advise.SqlAdvisorGetHintsFunction}.
+   * Tests {@link org.apache.calcite.sql.advise.SqlAdvisorGetHintsFunction}.
    */
   @Test public void testSqlAdvisorSchemaNames()
       throws SQLException, ClassNotFoundException {
     String res = adviseSql("select empid from \"emps\" e, ^");
     assertThat(res,
-        equalTo(
-            "id=; names=null; type=MATCH\n"
+        equalTo("id=; names=null; type=MATCH\n"
             + "id=(; names=[(]; type=KEYWORD\n"
             + "id=LATERAL; names=[LATERAL]; type=KEYWORD\n"
             + "id=TABLE; names=[TABLE]; type=KEYWORD\n"
@@ -362,28 +403,26 @@ public class JdbcTest {
 
   private String adviseSql(String sql) throws ClassNotFoundException,
       SQLException {
-    Class.forName("net.hydromatic.optiq.jdbc.Driver");
     Properties info = new Properties();
     info.put("lex", "JAVA");
     info.put("quoting", "DOUBLE_QUOTE");
     Connection connection =
         DriverManager.getConnection("jdbc:calcite:", info);
-    OptiqConnection optiqConnection =
-        connection.unwrap(OptiqConnection.class);
-    SchemaPlus rootSchema = optiqConnection.getRootSchema();
+    CalciteConnection calciteConnection =
+        connection.unwrap(CalciteConnection.class);
+    SchemaPlus rootSchema = calciteConnection.getRootSchema();
     rootSchema.add("hr", new ReflectiveSchema(new HrSchema()));
     SchemaPlus schema = rootSchema.add("s", new AbstractSchema());
-    optiqConnection.setSchema("hr");
+    calciteConnection.setSchema("hr");
     final TableFunction table =
         new SqlAdvisorGetHintsFunction();
     schema.add("get_hints", table);
-    PreparedStatement ps = connection.prepareStatement(
-        "select *\n"
+    PreparedStatement ps = connection.prepareStatement("select *\n"
         + "from table(\"s\".\"get_hints\"(?, ?)) as t(id, names, type)");
     SqlParserUtil.StringAndPos sap = SqlParserUtil.findPos(sql);
     ps.setString(1, sap.sql);
     ps.setInt(2, sap.cursor);
-    return OptiqAssert.toString(ps.executeQuery());
+    return CalciteAssert.toString(ps.executeQuery());
   }
 
   /**
@@ -391,27 +430,24 @@ public class JdbcTest {
    *
    * <p>The function ({@link #view(String)} has a return type
    * {@link Table} and the actual returned value implements
-   * {@link net.hydromatic.optiq.TranslatableTable}.
+   * {@link org.apache.calcite.schema.TranslatableTable}.
    */
   @Test public void testTableMacro()
       throws SQLException, ClassNotFoundException {
-    Class.forName("net.hydromatic.optiq.jdbc.Driver");
     Connection connection =
         DriverManager.getConnection("jdbc:calcite:");
-    OptiqConnection optiqConnection =
-        connection.unwrap(OptiqConnection.class);
-    SchemaPlus rootSchema = optiqConnection.getRootSchema();
+    CalciteConnection calciteConnection =
+        connection.unwrap(CalciteConnection.class);
+    SchemaPlus rootSchema = calciteConnection.getRootSchema();
     SchemaPlus schema = rootSchema.add("s", new AbstractSchema());
     final TableMacro tableMacro = TableMacroImpl.create(VIEW_METHOD);
     schema.add("View", tableMacro);
-    ResultSet resultSet = connection.createStatement().executeQuery(
-        "select *\n"
+    ResultSet resultSet = connection.createStatement().executeQuery("select *\n"
         + "from table(\"s\".\"View\"('(10), (20)')) as t(n)\n"
         + "where n < 15");
     // The call to "View('(10), (2)')" expands to 'values (1), (3), (10), (20)'.
-    assertThat(OptiqAssert.toString(resultSet),
-        equalTo(""
-            + "N=1\n"
+    assertThat(CalciteAssert.toString(resultSet),
+        equalTo("N=1\n"
             + "N=3\n"
             + "N=10\n"));
   }
@@ -441,9 +477,8 @@ public class JdbcTest {
   }
 
   private void checkTableMacroInModel(Class clazz) {
-    OptiqAssert.that()
-        .withModel(
-            "{\n"
+    CalciteAssert.that()
+        .withModel("{\n"
             + "  version: '1.0',\n"
             + "   schemas: [\n"
             + "     {\n"
@@ -456,7 +491,8 @@ public class JdbcTest {
             + "       ]\n"
             + "     }\n"
             + "   ]\n"
-            + "}").query("select * from table(\"adhoc\".\"View\"('(30)'))")
+            + "}")
+        .query("select * from table(\"adhoc\".\"View\"('(30)'))")
         .returns(""
             + "c=1\n"
             + "c=3\n"
@@ -469,7 +505,7 @@ public class JdbcTest {
   }
 
   /** A function that generates a table that generates a sequence of
-   * {@link net.hydromatic.optiq.test.JdbcTest.IntString} values. */
+   * {@link org.apache.calcite.test.JdbcTest.IntString} values. */
   public static QueryableTable generateStrings(final Integer count) {
     return new AbstractQueryableTable(IntString.class) {
       public RelDataType getRowType(RelDataTypeFactory typeFactory) {
@@ -538,8 +574,7 @@ public class JdbcTest {
       public Queryable<Object[]> asQueryable(QueryProvider queryProvider,
           SchemaPlus schema, String tableName) {
         final List<Object[]> table = new AbstractList<Object[]>() {
-          @Override
-          public Object[] get(int index) {
+          @Override public Object[] get(int index) {
             Object[] cur = new Object[ncol + 1];
             cur[0] = "row " + index;
             for (int j = 1; j <= ncol; j++) {
@@ -548,8 +583,7 @@ public class JdbcTest {
             return cur;
           }
 
-          @Override
-          public int size() {
+          @Override public int size() {
             return nrow;
           }
         };
@@ -609,27 +643,26 @@ public class JdbcTest {
     };
   }
 
-  /** Tests {@link net.hydromatic.avatica.Handler#onConnectionClose}
-   * and  {@link net.hydromatic.avatica.Handler#onStatementClose}. */
+  /** Tests {@link org.apache.calcite.avatica.Handler#onConnectionClose}
+   * and  {@link org.apache.calcite.avatica.Handler#onStatementClose}. */
   @Test public void testOnConnectionClose() throws Exception {
     final int[] closeCount = {0};
     final int[] statementCloseCount = {0};
     HandlerDriver.HANDLERS.set(
         new HandlerImpl() {
-          @Override
-          public void onConnectionClose(AvaticaConnection connection) {
+          @Override public void
+          onConnectionClose(AvaticaConnection connection) {
             ++closeCount[0];
             throw new RuntimeException();
           }
-          @Override
-          public void onStatementClose(AvaticaStatement statement) {
+          @Override public void onStatementClose(AvaticaStatement statement) {
             ++statementCloseCount[0];
             throw new RuntimeException();
           }
         });
     final HandlerDriver driver =
         new HandlerDriver();
-    OptiqConnection connection = (OptiqConnection)
+    CalciteConnection connection = (CalciteConnection)
         driver.connect("jdbc:calcite:", new Properties());
     SchemaPlus rootSchema = connection.getRootSchema();
     rootSchema.add("hr", new ReflectiveSchema(new HrSchema()));
@@ -679,15 +712,15 @@ public class JdbcTest {
       return;
     }
     final Driver driver = new Driver();
-    OptiqConnection connection = (OptiqConnection)
+    CalciteConnection connection = (CalciteConnection)
         driver.connect("jdbc:calcite:", new Properties());
     SchemaPlus rootSchema = connection.getRootSchema();
     rootSchema.add("hr", new ReflectiveSchema(new HrSchema()));
     connection.setSchema("hr");
     final Statement statement = connection.createStatement();
-    assertFalse((Boolean) OptiqAssert.call(statement, "isCloseOnCompletion"));
-    OptiqAssert.call(statement, "closeOnCompletion");
-    assertTrue((Boolean) OptiqAssert.call(statement, "isCloseOnCompletion"));
+    assertFalse((Boolean) CalciteAssert.call(statement, "isCloseOnCompletion"));
+    CalciteAssert.call(statement, "closeOnCompletion");
+    assertTrue((Boolean) CalciteAssert.call(statement, "isCloseOnCompletion"));
     final ResultSet resultSet =
         statement.executeQuery("select * from \"emps\"");
 
@@ -711,21 +744,20 @@ public class JdbcTest {
    * The example in the README.
    */
   @Test public void testReadme() throws ClassNotFoundException, SQLException {
-    Class.forName("net.hydromatic.optiq.jdbc.Driver");
     Connection connection = DriverManager.getConnection("jdbc:calcite:");
-    OptiqConnection optiqConnection =
-        connection.unwrap(OptiqConnection.class);
-    final SchemaPlus rootSchema = optiqConnection.getRootSchema();
+    CalciteConnection calciteConnection =
+        connection.unwrap(CalciteConnection.class);
+    final SchemaPlus rootSchema = calciteConnection.getRootSchema();
     rootSchema.add("hr", new ReflectiveSchema(new HrSchema()));
-    Statement statement = optiqConnection.createStatement();
-    ResultSet resultSet = statement.executeQuery(
-        "select d.\"deptno\", min(e.\"empid\")\n"
+    Statement statement = calciteConnection.createStatement();
+    ResultSet resultSet =
+        statement.executeQuery("select d.\"deptno\", min(e.\"empid\")\n"
             + "from \"hr\".\"emps\" as e\n"
             + "join \"hr\".\"depts\" as d\n"
             + "  on e.\"deptno\" = d.\"deptno\"\n"
             + "group by d.\"deptno\"\n"
             + "having count(*) > 1");
-    final String s = OptiqAssert.toString(resultSet);
+    final String s = CalciteAssert.toString(resultSet);
     assertThat(s, notNullValue());
     resultSet.close();
     statement.close();
@@ -735,7 +767,6 @@ public class JdbcTest {
   /** Test for {@link Driver#getPropertyInfo(String, Properties)}. */
   @Test public void testConnectionProperties() throws ClassNotFoundException,
       SQLException {
-    Class.forName("net.hydromatic.optiq.jdbc.Driver");
     java.sql.Driver driver = DriverManager.getDriver("jdbc:calcite:");
     final DriverPropertyInfo[] propertyInfo =
         driver.getPropertyInfo("jdbc:calcite:", new Properties());
@@ -752,18 +783,17 @@ public class JdbcTest {
    * Make sure that the properties look sane.
    */
   @Test public void testVersion() throws ClassNotFoundException, SQLException {
-    Class.forName("net.hydromatic.optiq.jdbc.Driver");
     Connection connection = DriverManager.getConnection("jdbc:calcite:");
-    OptiqConnection optiqConnection =
-        connection.unwrap(OptiqConnection.class);
-    final DatabaseMetaData metaData = optiqConnection.getMetaData();
+    CalciteConnection calciteConnection =
+        connection.unwrap(CalciteConnection.class);
+    final DatabaseMetaData metaData = calciteConnection.getMetaData();
     assertEquals("Calcite JDBC Driver", metaData.getDriverName());
 
     final String driverVersion = metaData.getDriverVersion();
     final int driverMajorVersion = metaData.getDriverMajorVersion();
     final int driverMinorVersion = metaData.getDriverMinorVersion();
     assertEquals(1, driverMajorVersion);
-    assertEquals(0, driverMinorVersion);
+    assertTrue(driverMinorVersion == 0); // will work for the next few releases
 
     assertEquals("Calcite", metaData.getDatabaseProductName());
     final String databaseProductVersion =
@@ -774,9 +804,9 @@ public class JdbcTest {
     assertEquals(driverMinorVersion, databaseMinorVersion);
 
     // Check how version is composed of major and minor version. Note that
-    // version is stored in pom.xml, then generated into
-    // net-hydromatic-optiq-jdbc.properties; major and minor version are
-    // found by splitting the version string.
+    // version is stored in pom.xml; major and minor version are
+    // stored in org-apache-calcite-jdbc.properties, but derived from
+    // version.major and version.minor in pom.xml.
     assertTrue(driverVersion.startsWith("1."));
     String[] split = driverVersion.split("\\.");
     assertTrue(split.length >= 2);
@@ -799,7 +829,7 @@ public class JdbcTest {
   /** Tests driver's implementation of {@link DatabaseMetaData#getColumns}. */
   @Test public void testMetaDataColumns()
       throws ClassNotFoundException, SQLException {
-    Connection connection = OptiqAssert.getConnection("hr", "foodmart");
+    Connection connection = CalciteAssert.getConnection("hr", "foodmart");
     DatabaseMetaData metaData = connection.getMetaData();
     ResultSet resultSet = metaData.getColumns(null, null, null, null);
     assertTrue(resultSet.next()); // there's something
@@ -819,7 +849,7 @@ public class JdbcTest {
    * It is empty but it should still have column definitions. */
   @Test public void testMetaDataPrimaryKeys()
       throws ClassNotFoundException, SQLException {
-    Connection connection = OptiqAssert.getConnection("hr", "foodmart");
+    Connection connection = CalciteAssert.getConnection("hr", "foodmart");
     DatabaseMetaData metaData = connection.getMetaData();
     ResultSet resultSet = metaData.getPrimaryKeys(null, null, null);
     assertFalse(resultSet.next()); // catalog never contains primary keys
@@ -833,7 +863,7 @@ public class JdbcTest {
   }
 
   /** Unit test for
-   * {@link net.hydromatic.optiq.jdbc.MetaImpl#likeToRegex(net.hydromatic.avatica.Meta.Pat)}. */
+   * {@link org.apache.calcite.jdbc.MetaImpl#likeToRegex(org.apache.calcite.avatica.Meta.Pat)}. */
   @Test public void testLikeToRegex() {
     checkLikeToRegex(true, "%", "abc");
     checkLikeToRegex(true, "abc", "abc");
@@ -865,11 +895,10 @@ public class JdbcTest {
   /** Tests driver's implementation of {@link DatabaseMetaData#getColumns}. */
   @Test public void testResultSetMetaData()
       throws ClassNotFoundException, SQLException {
-    Connection connection = OptiqAssert.getConnection("hr", "foodmart");
+    Connection connection = CalciteAssert.getConnection("hr", "foodmart");
     Statement statement = connection.createStatement();
     ResultSet resultSet =
-        statement.executeQuery(
-            "select \"empid\", \"deptno\" as x, 1 as y\n"
+        statement.executeQuery("select \"empid\", \"deptno\" as x, 1 as y\n"
             + "from \"hr\".\"emps\"");
     ResultSetMetaData metaData = resultSet.getMetaData();
     assertEquals(3, metaData.getColumnCount());
@@ -890,8 +919,8 @@ public class JdbcTest {
    * like to use them to check whether the connection is alive.
    */
   @Test public void testSimple() {
-    OptiqAssert.that()
-        .with(OptiqAssert.Config.FOODMART_CLONE)
+    CalciteAssert.that()
+        .with(CalciteAssert.Config.FOODMART_CLONE)
         .query("SELECT 1")
         .returns("EXPR$0=1\n");
   }
@@ -901,10 +930,10 @@ public class JdbcTest {
     // JDBC 3.0 specification: "Column names supplied to getter methods are case
     // insensitive. If a select list contains the same column more than once,
     // the first instance of the column will be returned."
-    OptiqAssert.that()
+    CalciteAssert.that()
         .doWithConnection(
-            new Function<OptiqConnection, Object>() {
-              public Object apply(OptiqConnection c) {
+            new Function<CalciteConnection, Object>() {
+              public Object apply(CalciteConnection c) {
                 try {
                   Statement s = c.createStatement();
                   ResultSet rs =
@@ -960,8 +989,8 @@ public class JdbcTest {
 
   @Test public void testCloneSchema()
       throws ClassNotFoundException, SQLException {
-    final OptiqConnection connection =
-        OptiqAssert.getConnection(OptiqAssert.SchemaSpec.JDBC_FOODMART);
+    final CalciteConnection connection =
+        CalciteAssert.getConnection(CalciteAssert.SchemaSpec.JDBC_FOODMART);
     final SchemaPlus rootSchema = connection.getRootSchema();
     final SchemaPlus foodmart = rootSchema.getSubSchema("foodmart");
     rootSchema.add("foodmart2", new CloneSchema(foodmart));
@@ -976,10 +1005,9 @@ public class JdbcTest {
   }
 
   @Test public void testCloneGroupBy() {
-    OptiqAssert.that()
-        .with(OptiqAssert.Config.FOODMART_CLONE)
-        .query(
-            "select \"the_year\", count(*) as c, min(\"the_month\") as m\n"
+    CalciteAssert.that()
+        .with(CalciteAssert.Config.FOODMART_CLONE)
+        .query("select \"the_year\", count(*) as c, min(\"the_month\") as m\n"
             + "from \"foodmart2\".\"time_by_day\"\n"
             + "group by \"the_year\"\n"
             + "order by 1, 2")
@@ -990,12 +1018,11 @@ public class JdbcTest {
 
   @Ignore
   @Test public void testCloneGroupBy2() {
-    OptiqAssert.that()
-        .with(OptiqAssert.Config.FOODMART_CLONE)
+    CalciteAssert.that()
+        .with(CalciteAssert.Config.FOODMART_CLONE)
         .query(
             "select \"time_by_day\".\"the_year\" as \"c0\", \"time_by_day\".\"quarter\" as \"c1\", \"product_class\".\"product_family\" as \"c2\", sum(\"sales_fact_1997\".\"unit_sales\") as \"m0\" from \"time_by_day\" as \"time_by_day\", \"sales_fact_1997\" as \"sales_fact_1997\", \"product_class\" as \"product_class\", \"product\" as \"product\" where \"sales_fact_1997\".\"time_id\" = \"time_by_day\".\"time_id\" and \"time_by_day\".\"the_year\" = 1997 and \"sales_fact_1997\".\"product_id\" = \"product\".\"product_id\" and \"product\".\"product_class_id\" = \"product_class\".\"product_class_id\" group by \"time_by_day\".\"the_year\", \"time_by_day\".\"quarter\", \"product_class\".\"product_family\"")
-        .returns(
-            "c0=1997; c1=Q2; c2=Drink; m0=5895.0000\n"
+        .returns("c0=1997; c1=Q2; c2=Drink; m0=5895.0000\n"
             + "c0=1997; c1=Q1; c2=Food; m0=47809.0000\n"
             + "c0=1997; c1=Q3; c2=Drink; m0=6065.0000\n"
             + "c0=1997; c1=Q4; c2=Drink; m0=6661.0000\n"
@@ -1012,38 +1039,36 @@ public class JdbcTest {
   /** Tests plan for a query with 4 tables, 3 joins. */
   @Ignore
   @Test public void testCloneGroupBy2Plan() {
-    OptiqAssert.that()
-        .with(OptiqAssert.Config.FOODMART_CLONE)
+    CalciteAssert.that()
+        .with(CalciteAssert.Config.FOODMART_CLONE)
         .query(
             "explain plan for select \"time_by_day\".\"the_year\" as \"c0\", \"time_by_day\".\"quarter\" as \"c1\", \"product_class\".\"product_family\" as \"c2\", sum(\"sales_fact_1997\".\"unit_sales\") as \"m0\" from \"time_by_day\" as \"time_by_day\", \"sales_fact_1997\" as \"sales_fact_1997\", \"product_class\" as \"product_class\", \"product\" as \"product\" where \"sales_fact_1997\".\"time_id\" = \"time_by_day\".\"time_id\" and \"time_by_day\".\"the_year\" = 1997 and \"sales_fact_1997\".\"product_id\" = \"product\".\"product_id\" and \"product\".\"product_class_id\" = \"product_class\".\"product_class_id\" group by \"time_by_day\".\"the_year\", \"time_by_day\".\"quarter\", \"product_class\".\"product_family\"")
-        .returns(
-            "PLAN=EnumerableAggregateRel(group=[{0, 1, 2}], m0=[SUM($3)])\n"
-            + "  EnumerableCalcRel(expr#0..37=[{inputs}], c0=[$t9], c1=[$t13], c2=[$t4], unit_sales=[$t22])\n"
-            + "    EnumerableJoinRel(condition=[=($23, $0)], joinType=[inner])\n"
-            + "      EnumerableTableAccessRel(table=[[foodmart2, product_class]])\n"
-            + "      EnumerableJoinRel(condition=[=($10, $19)], joinType=[inner])\n"
-            + "        EnumerableJoinRel(condition=[=($11, $0)], joinType=[inner])\n"
-            + "          EnumerableCalcRel(expr#0..9=[{inputs}], expr#10=[CAST($t4):INTEGER], expr#11=[1997], expr#12=[=($t10, $t11)], proj#0..9=[{exprs}], $condition=[$t12])\n"
-            + "            EnumerableTableAccessRel(table=[[foodmart2, time_by_day]])\n"
-            + "          EnumerableTableAccessRel(table=[[foodmart2, sales_fact_1997]])\n"
-            + "        EnumerableTableAccessRel(table=[[foodmart2, product]])\n"
+        .returns("PLAN=EnumerableAggregate(group=[{0, 1, 2}], m0=[SUM($3)])\n"
+            + "  EnumerableCalc(expr#0..37=[{inputs}], c0=[$t9], c1=[$t13], c2=[$t4], unit_sales=[$t22])\n"
+            + "    EnumerableJoin(condition=[=($23, $0)], joinType=[inner])\n"
+            + "      EnumerableTableScan(table=[[foodmart2, product_class]])\n"
+            + "      EnumerableJoin(condition=[=($10, $19)], joinType=[inner])\n"
+            + "        EnumerableJoin(condition=[=($11, $0)], joinType=[inner])\n"
+            + "          EnumerableCalc(expr#0..9=[{inputs}], expr#10=[CAST($t4):INTEGER], expr#11=[1997], expr#12=[=($t10, $t11)], proj#0..9=[{exprs}], $condition=[$t12])\n"
+            + "            EnumerableTableScan(table=[[foodmart2, time_by_day]])\n"
+            + "          EnumerableTableScan(table=[[foodmart2, sales_fact_1997]])\n"
+            + "        EnumerableTableScan(table=[[foodmart2, product]])\n"
             + "\n");
   }
 
   @Test public void testOrderByCase() {
-    OptiqAssert.that()
-        .with(OptiqAssert.Config.FOODMART_CLONE)
+    CalciteAssert.that()
+        .with(CalciteAssert.Config.FOODMART_CLONE)
         .query(
             "select \"time_by_day\".\"the_year\" as \"c0\" from \"time_by_day\" as \"time_by_day\" group by \"time_by_day\".\"the_year\" order by CASE WHEN \"time_by_day\".\"the_year\" IS NULL THEN 1 ELSE 0 END, \"time_by_day\".\"the_year\" ASC")
-        .returns(
-            "c0=1997\n"
+        .returns("c0=1997\n"
             + "c0=1998\n");
   }
 
   /** Just short of bushy. */
   @Test public void testAlmostBushy() {
-    OptiqAssert.that()
-        .with(OptiqAssert.Config.FOODMART_CLONE)
+    CalciteAssert.that()
+        .with(CalciteAssert.Config.FOODMART_CLONE)
         .query("select *\n"
             + "from \"sales_fact_1997\" as s\n"
             + "  join \"customer\" as c using (\"customer_id\")\n"
@@ -1051,42 +1076,41 @@ public class JdbcTest {
             + "where c.\"city\" = 'San Francisco'\n"
             + "and p.\"brand_name\" = 'Washington'")
         .explainMatches("including all attributes ",
-            OptiqAssert.checkMaskedResultContains(""
-                + "EnumerableJoinRel(condition=[=($0, $38)], joinType=[inner]): rowcount = 7.050660528307499E8, cumulative cost = {1.0640240206183146E9 rows, 777302.0 cpu, 0.0 io}\n"
-                + "  EnumerableJoinRel(condition=[=($2, $8)], joinType=[inner]): rowcount = 2.0087351932499997E7, cumulative cost = {2.117504619375143E7 rows, 724261.0 cpu, 0.0 io}\n"
-                + "    EnumerableTableAccessRel(table=[[foodmart2, sales_fact_1997]]): rowcount = 86837.0, cumulative cost = {86837.0 rows, 86838.0 cpu, 0.0 io}\n"
-                + "    EnumerableCalcRel(expr#0..28=[{inputs}], expr#29=['San Francisco'], expr#30=[=($t9, $t29)], proj#0..28=[{exprs}], $condition=[$t30]): rowcount = 1542.1499999999999, cumulative cost = {11823.15 rows, 637423.0 cpu, 0.0 io}\n"
-                + "      EnumerableTableAccessRel(table=[[foodmart2, customer]]): rowcount = 10281.0, cumulative cost = {10281.0 rows, 10282.0 cpu, 0.0 io}\n"
-                + "  EnumerableCalcRel(expr#0..14=[{inputs}], expr#15=['Washington'], expr#16=[=($t2, $t15)], proj#0..14=[{exprs}], $condition=[$t16]): rowcount = 234.0, cumulative cost = {1794.0 rows, 53041.0 cpu, 0.0 io}\n"
-                + "    EnumerableTableAccessRel(table=[[foodmart2, product]]): rowcount = 1560.0, cumulative cost = {1560.0 rows, 1561.0 cpu, 0.0 io}\n"));
+            CalciteAssert.checkMaskedResultContains(""
+                + "EnumerableJoin(condition=[=($0, $38)], joinType=[inner]): rowcount = 7.050660528307499E8, cumulative cost = {1.0640240206183146E9 rows, 777302.0 cpu, 0.0 io}\n"
+                + "  EnumerableJoin(condition=[=($2, $8)], joinType=[inner]): rowcount = 2.0087351932499997E7, cumulative cost = {2.117504619375143E7 rows, 724261.0 cpu, 0.0 io}\n"
+                + "    EnumerableTableScan(table=[[foodmart2, sales_fact_1997]]): rowcount = 86837.0, cumulative cost = {86837.0 rows, 86838.0 cpu, 0.0 io}\n"
+                + "    EnumerableCalc(expr#0..28=[{inputs}], expr#29=['San Francisco'], expr#30=[=($t9, $t29)], proj#0..28=[{exprs}], $condition=[$t30]): rowcount = 1542.1499999999999, cumulative cost = {11823.15 rows, 637423.0 cpu, 0.0 io}\n"
+                + "      EnumerableTableScan(table=[[foodmart2, customer]]): rowcount = 10281.0, cumulative cost = {10281.0 rows, 10282.0 cpu, 0.0 io}\n"
+                + "  EnumerableCalc(expr#0..14=[{inputs}], expr#15=['Washington'], expr#16=[=($t2, $t15)], proj#0..14=[{exprs}], $condition=[$t16]): rowcount = 234.0, cumulative cost = {1794.0 rows, 53041.0 cpu, 0.0 io}\n"
+                + "    EnumerableTableScan(table=[[foodmart2, product]]): rowcount = 1560.0, cumulative cost = {1560.0 rows, 1561.0 cpu, 0.0 io}\n"));
   }
 
   /** Tests a query whose best plan is a bushy join.
    * First join sales_fact_1997 to customer;
    * in parallel join product to product_class;
    * then join the results. */
-  @Ignore("extremely slow - a bit better if you disable MergeProjectRule")
+  @Ignore("extremely slow - a bit better if you disable ProjectMergeRule")
   @Test public void testBushy() {
-    OptiqAssert.that()
-      .with(OptiqAssert.Config.FOODMART_CLONE)
-      .query(
-        "select *\n"
-        + "from \"sales_fact_1997\" as s\n"
-        + "  join \"customer\" as c using (\"customer_id\")\n"
-        + "  join \"product\" as p using (\"product_id\")\n"
-        + "  join \"product_class\" as pc using (\"product_class_id\")\n"
-        + "where c.\"city\" = 'San Francisco'\n"
-        + "and pc.\"product_department\" = 'Snacks'\n")
+    CalciteAssert.that()
+      .with(CalciteAssert.Config.FOODMART_CLONE)
+      .query("select *\n"
+          + "from \"sales_fact_1997\" as s\n"
+          + "  join \"customer\" as c using (\"customer_id\")\n"
+          + "  join \"product\" as p using (\"product_id\")\n"
+          + "  join \"product_class\" as pc using (\"product_class_id\")\n"
+          + "where c.\"city\" = 'San Francisco'\n"
+          + "and pc.\"product_department\" = 'Snacks'\n")
         .explainMatches("including all attributes ",
-            OptiqAssert.checkMaskedResultContains(""
+            CalciteAssert.checkMaskedResultContains(""
                 + "EnumerableCalcRel(expr#0..56=[{inputs}], expr#57=['San Francisco'], expr#58=[=($t9, $t57)], expr#59=['Snacks'], expr#60=[=($t32, $t59)], expr#61=[AND($t58, $t60)], product_id=[$t49], time_id=[$t50], customer_id=[$t51], promotion_id=[$t52], store_id=[$t53], store_sales=[$t54], store_cost=[$t55], unit_sales=[$t56], customer_id0=[$t0], account_num=[$t1], lname=[$t2], fname=[$t3], mi=[$t4], address1=[$t5], address2=[$t6], address3=[$t7], address4=[$t8], city=[$t9], state_province=[$t10], postal_code=[$t11], country=[$t12], customer_region_id=[$t13], phone1=[$t14], phone2=[$t15], birthdate=[$t16], marital_status=[$t17], yearly_income=[$t18], gender=[$t19], total_children=[$t20], num_children_at_home=[$t21], education=[$t22], date_accnt_opened=[$t23], member_card=[$t24], occupation=[$t25], houseowner=[$t26], num_cars_owned=[$t27], fullname=[$t28], product_class_id=[$t34], product_id0=[$t35], brand_name=[$t36], product_name=[$t37], SKU=[$t38], SRP=[$t39], gross_weight=[$
 t40], net_weight=[$t41], recyclable_package=[$t42], low_fat=[$t43], units_per_case=[$t44], cases_per_pallet=[$t45], shelf_width=[$t46], shelf_height=[$t47], shelf_depth=[$t48], product_class_id0=[$t29], product_subcategory=[$t30], product_category=[$t31], product_department=[$t32], product_family=[$t33], $condition=[$t61]): rowcount = 1953.8325, cumulative cost = {728728.1144018068 rows, 1.0519232E7 cpu, 0.0 io}\n"
                 + "  EnumerableJoinRel(condition=[=($51, $0)], joinType=[inner]): rowcount = 86837.0, cumulative cost = {726774.2819018068 rows, 98792.0 cpu, 0.0 io}\n"
-                + "    EnumerableTableAccessRel(table=[[foodmart2, customer]]): rowcount = 10281.0, cumulative cost = {10281.0 rows, 10282.0 cpu, 0.0 io}\n"
+                + "    EnumerableTableScan(table=[[foodmart2, customer]]): rowcount = 10281.0, cumulative cost = {10281.0 rows, 10282.0 cpu, 0.0 io}\n"
                 + "    EnumerableJoinRel(condition=[=($5, $0)], joinType=[inner]): rowcount = 86837.0, cumulative cost = {447842.86095661717 rows, 88510.0 cpu, 0.0 io}\n"
-                + "      EnumerableTableAccessRel(table=[[foodmart2, product_class]]): rowcount = 110.0, cumulative cost = {110.0 rows, 111.0 cpu, 0.0 io}\n"
+                + "      EnumerableTableScan(table=[[foodmart2, product_class]]): rowcount = 110.0, cumulative cost = {110.0 rows, 111.0 cpu, 0.0 io}\n"
                 + "      EnumerableJoinRel(condition=[=($15, $1)], joinType=[inner]): rowcount = 86837.0, cumulative cost = {273541.80811638 rows, 88399.0 cpu, 0.0 io}\n"
-                + "        EnumerableTableAccessRel(table=[[foodmart2, product]]): rowcount = 1560.0, cumulative cost = {1560.0 rows, 1561.0 cpu, 0.0 io}\n"
-                + "        EnumerableTableAccessRel(table=[[foodmart2, sales_fact_1997]]): rowcount = 86837.0, cumulative cost = {86837.0 rows, 86838.0 cpu, 0.0 io}\n"));
+                + "        EnumerableTableScan(table=[[foodmart2, product]]): rowcount = 1560.0, cumulative cost = {1560.0 rows, 1561.0 cpu, 0.0 io}\n"
+                + "        EnumerableTableScan(table=[[foodmart2, sales_fact_1997]]): rowcount = 86837.0, cumulative cost = {86837.0 rows, 86838.0 cpu, 0.0 io}\n"));
   }
 
   private static final String[] QUERIES = {
@@ -1132,7 +1156,7 @@ public class JdbcTest {
     "EXPR$0=86805\n",
     "select \"time_by_day\".\"the_year\" as \"c0\" from \"time_by_day\" as \"time_by_day\" group by \"time_by_day\".\"the_year\" order by \"time_by_day\".\"the_year\" ASC",
     "c0=1997\n"
-      + "c0=1998\n",
+        + "c0=1998\n",
     "select \"store\".\"store_country\" as \"c0\" from \"store\" as \"store\" where UPPER(\"store\".\"store_country\") = UPPER('USA') group by \"store\".\"store_country\" order by \"store\".\"store_country\" ASC",
     "c0=USA\n",
     "select \"store\".\"store_state\" as \"c0\" from \"store\" as \"store\" where (\"store\".\"store_country\" = 'USA') and UPPER(\"store\".\"store_state\") = UPPER('CA') group by \"store\".\"store_state\" order by \"store\".\"store_state\" ASC",
@@ -1159,223 +1183,223 @@ public class JdbcTest {
     "c0=No Media\n",
     "select \"promotion\".\"media_type\" as \"c0\" from \"promotion\" as \"promotion\" group by \"promotion\".\"media_type\" order by \"promotion\".\"media_type\" ASC",
     "c0=Bulk Mail\n"
-      + "c0=Cash Register Handout\n"
-      + "c0=Daily Paper\n"
-      + "c0=Daily Paper, Radio\n"
-      + "c0=Daily Paper, Radio, TV\n"
-      + "c0=In-Store Coupon\n"
-      + "c0=No Media\n"
-      + "c0=Product Attachment\n"
-      + "c0=Radio\n"
-      + "c0=Street Handout\n"
-      + "c0=Sunday Paper\n"
-      + "c0=Sunday Paper, Radio\n"
-      + "c0=Sunday Paper, Radio, TV\n"
-      + "c0=TV\n",
+        + "c0=Cash Register Handout\n"
+        + "c0=Daily Paper\n"
+        + "c0=Daily Paper, Radio\n"
+        + "c0=Daily Paper, Radio, TV\n"
+        + "c0=In-Store Coupon\n"
+        + "c0=No Media\n"
+        + "c0=Product Attachment\n"
+        + "c0=Radio\n"
+        + "c0=Street Handout\n"
+        + "c0=Sunday Paper\n"
+        + "c0=Sunday Paper, Radio\n"
+        + "c0=Sunday Paper, Radio, TV\n"
+        + "c0=TV\n",
     "select count(distinct \"the_year\") from \"time_by_day\"",
     "EXPR$0=2\n",
     "select \"time_by_day\".\"the_year\" as \"c0\", sum(\"sales_fact_1997\".\"unit_sales\") as \"m0\" from \"time_by_day\" as \"time_by_day\", \"sales_fact_1997\" as \"sales_fact_1997\" where \"sales_fact_1997\".\"time_id\" = \"time_by_day\".\"time_id\" and \"time_by_day\".\"the_year\" = 1997 group by \"time_by_day\".\"the_year\"",
     "c0=1997; m0=266773.0000\n",
     "select \"time_by_day\".\"the_year\" as \"c0\", \"promotion\".\"media_type\" as \"c1\", sum(\"sales_fact_1997\".\"unit_sales\") as \"m0\" from \"time_by_day\" as \"time_by_day\", \"sales_fact_1997\" as \"sales_fact_1997\", \"promotion\" as \"promotion\" where \"sales_fact_1997\".\"time_id\" = \"time_by_day\".\"time_id\" and \"time_by_day\".\"the_year\" = 1997 and \"sales_fact_1997\".\"promotion_id\" = \"promotion\".\"promotion_id\" group by \"time_by_day\".\"the_year\", \"promotion\".\"media_type\"",
     "c0=1997; c1=Bulk Mail; m0=4320.0000\n"
-      + "c0=1997; c1=Radio; m0=2454.0000\n"
-      + "c0=1997; c1=Street Handout; m0=5753.0000\n"
-      + "c0=1997; c1=TV; m0=3607.0000\n"
-      + "c0=1997; c1=No Media; m0=195448.0000\n"
-      + "c0=1997; c1=In-Store Coupon; m0=3798.0000\n"
-      + "c0=1997; c1=Sunday Paper, Radio, TV; m0=2726.0000\n"
-      + "c0=1997; c1=Product Attachment; m0=7544.0000\n"
-      + "c0=1997; c1=Daily Paper; m0=7738.0000\n"
-      + "c0=1997; c1=Cash Register Handout; m0=6697.0000\n"
-      + "c0=1997; c1=Daily Paper, Radio; m0=6891.0000\n"
-      + "c0=1997; c1=Daily Paper, Radio, TV; m0=9513.0000\n"
-      + "c0=1997; c1=Sunday Paper, Radio; m0=5945.0000\n"
-      + "c0=1997; c1=Sunday Paper; m0=4339.0000\n",
+        + "c0=1997; c1=Radio; m0=2454.0000\n"
+        + "c0=1997; c1=Street Handout; m0=5753.0000\n"
+        + "c0=1997; c1=TV; m0=3607.0000\n"
+        + "c0=1997; c1=No Media; m0=195448.0000\n"
+        + "c0=1997; c1=In-Store Coupon; m0=3798.0000\n"
+        + "c0=1997; c1=Sunday Paper, Radio, TV; m0=2726.0000\n"
+        + "c0=1997; c1=Product Attachment; m0=7544.0000\n"
+        + "c0=1997; c1=Daily Paper; m0=7738.0000\n"
+        + "c0=1997; c1=Cash Register Handout; m0=6697.0000\n"
+        + "c0=1997; c1=Daily Paper, Radio; m0=6891.0000\n"
+        + "c0=1997; c1=Daily Paper, Radio, TV; m0=9513.0000\n"
+        + "c0=1997; c1=Sunday Paper, Radio; m0=5945.0000\n"
+        + "c0=1997; c1=Sunday Paper; m0=4339.0000\n",
     "select \"store\".\"store_country\" as \"c0\", sum(\"inventory_fact_1997\".\"supply_time\") as \"m0\" from \"store\" as \"store\", \"inventory_fact_1997\" as \"inventory_fact_1997\" where \"inventory_fact_1997\".\"store_id\" = \"store\".\"store_id\" group by \"store\".\"store_country\"",
     "c0=USA; m0=10425\n",
     "select \"sn\".\"desc\" as \"c0\" from (SELECT * FROM (VALUES (1, 'SameName')) AS \"t\" (\"id\", \"desc\")) as \"sn\" group by \"sn\".\"desc\" order by \"sn\".\"desc\" ASC NULLS LAST",
     "c0=SameName\n",
     "select \"the_year\", count(*) as c, min(\"the_month\") as m\n"
-      + "from \"foodmart2\".\"time_by_day\"\n"
-      + "group by \"the_year\"\n"
-      + "order by 1, 2",
+        + "from \"foodmart2\".\"time_by_day\"\n"
+        + "group by \"the_year\"\n"
+        + "order by 1, 2",
     "the_year=1997; C=365; M=April\n"
-      + "the_year=1998; C=365; M=April\n",
+        + "the_year=1998; C=365; M=April\n",
     "select\n"
-      + " \"store\".\"store_state\" as \"c0\",\n"
-      + " \"time_by_day\".\"the_year\" as \"c1\",\n"
-      + " sum(\"sales_fact_1997\".\"unit_sales\") as \"m0\",\n"
-      + " sum(\"sales_fact_1997\".\"store_sales\") as \"m1\"\n"
-      + "from \"store\" as \"store\",\n"
-      + " \"sales_fact_1997\" as \"sales_fact_1997\",\n"
-      + " \"time_by_day\" as \"time_by_day\"\n"
-      + "where \"sales_fact_1997\".\"store_id\" = \"store\".\"store_id\"\n"
-      + "and \"store\".\"store_state\" in ('DF', 'WA')\n"
-      + "and \"sales_fact_1997\".\"time_id\" = \"time_by_day\".\"time_id\"\n"
-      + "and \"time_by_day\".\"the_year\" = 1997\n"
-      + "group by \"store\".\"store_state\", \"time_by_day\".\"the_year\"",
+        + " \"store\".\"store_state\" as \"c0\",\n"
+        + " \"time_by_day\".\"the_year\" as \"c1\",\n"
+        + " sum(\"sales_fact_1997\".\"unit_sales\") as \"m0\",\n"
+        + " sum(\"sales_fact_1997\".\"store_sales\") as \"m1\"\n"
+        + "from \"store\" as \"store\",\n"
+        + " \"sales_fact_1997\" as \"sales_fact_1997\",\n"
+        + " \"time_by_day\" as \"time_by_day\"\n"
+        + "where \"sales_fact_1997\".\"store_id\" = \"store\".\"store_id\"\n"
+        + "and \"store\".\"store_state\" in ('DF', 'WA')\n"
+        + "and \"sales_fact_1997\".\"time_id\" = \"time_by_day\".\"time_id\"\n"
+        + "and \"time_by_day\".\"the_year\" = 1997\n"
+        + "group by \"store\".\"store_state\", \"time_by_day\".\"the_year\"",
     "c0=WA; c1=1997; m0=124366.0000; m1=263793.2200\n",
     "select count(distinct \"product_id\") from \"product\"",
     "EXPR$0=1560\n",
     "select \"store\".\"store_name\" as \"c0\",\n"
-      + " \"time_by_day\".\"the_year\" as \"c1\",\n"
-      + " sum(\"sales_fact_1997\".\"store_sales\") as \"m0\"\n"
-      + "from \"store\" as \"store\",\n"
-      + " \"sales_fact_1997\" as \"sales_fact_1997\",\n"
-      + " \"time_by_day\" as \"time_by_day\"\n"
-      + "where \"sales_fact_1997\".\"store_id\" = \"store\".\"store_id\"\n"
-      + "and \"store\".\"store_name\" in ('Store 1', 'Store 10', 'Store 11', 'Store 15', 'Store 16', 'Store 24', 'Store 3', 'Store 7')\n"
-      + "and \"sales_fact_1997\".\"time_id\" = \"time_by_day\".\"time_id\"\n"
-      + "and \"time_by_day\".\"the_year\" = 1997\n"
-      + "group by \"store\".\"store_name\",\n"
-      + " \"time_by_day\".\"the_year\"\n",
+        + " \"time_by_day\".\"the_year\" as \"c1\",\n"
+        + " sum(\"sales_fact_1997\".\"store_sales\") as \"m0\"\n"
+        + "from \"store\" as \"store\",\n"
+        + " \"sales_fact_1997\" as \"sales_fact_1997\",\n"
+        + " \"time_by_day\" as \"time_by_day\"\n"
+        + "where \"sales_fact_1997\".\"store_id\" = \"store\".\"store_id\"\n"
+        + "and \"store\".\"store_name\" in ('Store 1', 'Store 10', 'Store 11', 'Store 15', 'Store 16', 'Store 24', 'Store 3', 'Store 7')\n"
+        + "and \"sales_fact_1997\".\"time_id\" = \"time_by_day\".\"time_id\"\n"
+        + "and \"time_by_day\".\"the_year\" = 1997\n"
+        + "group by \"store\".\"store_name\",\n"
+        + " \"time_by_day\".\"the_year\"\n",
     "c0=Store 7; c1=1997; m0=54545.2800\n"
-      + "c0=Store 24; c1=1997; m0=54431.1400\n"
-      + "c0=Store 16; c1=1997; m0=49634.4600\n"
-      + "c0=Store 3; c1=1997; m0=52896.3000\n"
-      + "c0=Store 15; c1=1997; m0=52644.0700\n"
-      + "c0=Store 11; c1=1997; m0=55058.7900\n",
+        + "c0=Store 24; c1=1997; m0=54431.1400\n"
+        + "c0=Store 16; c1=1997; m0=49634.4600\n"
+        + "c0=Store 3; c1=1997; m0=52896.3000\n"
+        + "c0=Store 15; c1=1997; m0=52644.0700\n"
+        + "c0=Store 11; c1=1997; m0=55058.7900\n",
     "select \"customer\".\"yearly_income\" as \"c0\","
-      + " \"customer\".\"education\" as \"c1\" \n"
-      + "from \"customer\" as \"customer\",\n"
-      + " \"sales_fact_1997\" as \"sales_fact_1997\"\n"
-      + "where \"sales_fact_1997\".\"customer_id\" = \"customer\".\"customer_id\"\n"
-      + " and ((not (\"customer\".\"yearly_income\" in ('$10K - $30K', '$50K - $70K'))\n"
-      + " or (\"customer\".\"yearly_income\" is null)))\n"
-      + "group by \"customer\".\"yearly_income\",\n"
-      + " \"customer\".\"education\"\n"
-      + "order by \"customer\".\"yearly_income\" ASC NULLS LAST,\n"
-      + " \"customer\".\"education\" ASC NULLS LAST",
+        + " \"customer\".\"education\" as \"c1\" \n"
+        + "from \"customer\" as \"customer\",\n"
+        + " \"sales_fact_1997\" as \"sales_fact_1997\"\n"
+        + "where \"sales_fact_1997\".\"customer_id\" = \"customer\".\"customer_id\"\n"
+        + " and ((not (\"customer\".\"yearly_income\" in ('$10K - $30K', '$50K - $70K'))\n"
+        + " or (\"customer\".\"yearly_income\" is null)))\n"
+        + "group by \"customer\".\"yearly_income\",\n"
+        + " \"customer\".\"education\"\n"
+        + "order by \"customer\".\"yearly_income\" ASC NULLS LAST,\n"
+        + " \"customer\".\"education\" ASC NULLS LAST",
     "c0=$110K - $130K; c1=Bachelors Degree\n"
-      + "c0=$110K - $130K; c1=Graduate Degree\n"
-      + "c0=$110K - $130K; c1=High School Degree\n"
-      + "c0=$110K - $130K; c1=Partial College\n"
-      + "c0=$110K - $130K; c1=Partial High School\n"
-      + "c0=$130K - $150K; c1=Bachelors Degree\n"
-      + "c0=$130K - $150K; c1=Graduate Degree\n"
-      + "c0=$130K - $150K; c1=High School Degree\n"
-      + "c0=$130K - $150K; c1=Partial College\n"
-      + "c0=$130K - $150K; c1=Partial High School\n"
-      + "c0=$150K +; c1=Bachelors Degree\n"
-      + "c0=$150K +; c1=Graduate Degree\n"
-      + "c0=$150K +; c1=High School Degree\n"
-      + "c0=$150K +; c1=Partial College\n"
-      + "c0=$150K +; c1=Partial High School\n"
-      + "c0=$30K - $50K; c1=Bachelors Degree\n"
-      + "c0=$30K - $50K; c1=Graduate Degree\n"
-      + "c0=$30K - $50K; c1=High School Degree\n"
-      + "c0=$30K - $50K; c1=Partial College\n"
-      + "c0=$30K - $50K; c1=Partial High School\n"
-      + "c0=$70K - $90K; c1=Bachelors Degree\n"
-      + "c0=$70K - $90K; c1=Graduate Degree\n"
-      + "c0=$70K - $90K; c1=High School Degree\n"
-      + "c0=$70K - $90K; c1=Partial College\n"
-      + "c0=$70K - $90K; c1=Partial High School\n"
-      + "c0=$90K - $110K; c1=Bachelors Degree\n"
-      + "c0=$90K - $110K; c1=Graduate Degree\n"
-      + "c0=$90K - $110K; c1=High School Degree\n"
-      + "c0=$90K - $110K; c1=Partial College\n"
-      + "c0=$90K - $110K; c1=Partial High School\n",
+        + "c0=$110K - $130K; c1=Graduate Degree\n"
+        + "c0=$110K - $130K; c1=High School Degree\n"
+        + "c0=$110K - $130K; c1=Partial College\n"
+        + "c0=$110K - $130K; c1=Partial High School\n"
+        + "c0=$130K - $150K; c1=Bachelors Degree\n"
+        + "c0=$130K - $150K; c1=Graduate Degree\n"
+        + "c0=$130K - $150K; c1=High School Degree\n"
+        + "c0=$130K - $150K; c1=Partial College\n"
+        + "c0=$130K - $150K; c1=Partial High School\n"
+        + "c0=$150K +; c1=Bachelors Degree\n"
+        + "c0=$150K +; c1=Graduate Degree\n"
+        + "c0=$150K +; c1=High School Degree\n"
+        + "c0=$150K +; c1=Partial College\n"
+        + "c0=$150K +; c1=Partial High School\n"
+        + "c0=$30K - $50K; c1=Bachelors Degree\n"
+        + "c0=$30K - $50K; c1=Graduate Degree\n"
+        + "c0=$30K - $50K; c1=High School Degree\n"
+        + "c0=$30K - $50K; c1=Partial College\n"
+        + "c0=$30K - $50K; c1=Partial High School\n"
+        + "c0=$70K - $90K; c1=Bachelors Degree\n"
+        + "c0=$70K - $90K; c1=Graduate Degree\n"
+        + "c0=$70K - $90K; c1=High School Degree\n"
+        + "c0=$70K - $90K; c1=Partial College\n"
+        + "c0=$70K - $90K; c1=Partial High School\n"
+        + "c0=$90K - $110K; c1=Bachelors Degree\n"
+        + "c0=$90K - $110K; c1=Graduate Degree\n"
+        + "c0=$90K - $110K; c1=High School Degree\n"
+        + "c0=$90K - $110K; c1=Partial College\n"
+        + "c0=$90K - $110K; c1=Partial High School\n",
     "ignore:select \"time_by_day\".\"the_year\" as \"c0\", \"product_class\".\"product_family\" as \"c1\", \"customer\".\"state_province\" as \"c2\", \"customer\".\"city\" as \"c3\", sum(\"sales_fact_1997\".\"unit_sales\") as \"m0\" from \"time_by_day\" as \"time_by_day\", \"sales_fact_1997\" as \"sales_fact_1997\", \"product_class\" as \"product_class\", \"product\" as \"product\", \"customer\" as \"customer\" where \"sales_fact_1997\".\"time_id\" = \"time_by_day\".\"time_id\" and \"time_by_day\".\"the_year\" = 1997 and \"sales_fact_1997\".\"product_id\" = \"product\".\"product_id\" and \"product\".\"product_class_id\" = \"product_class\".\"product_class_id\" and \"product_class\".\"product_family\" = 'Drink' and \"sales_fact_1997\".\"customer_id\" = \"customer\".\"customer_id\" and \"customer\".\"state_province\" = 'WA' and \"customer\".\"city\" in ('Anacortes', 'Ballard', 'Bellingham', 'Bremerton', 'Burien', 'Edmonds', 'Everett', 'Issaquah', 'Kirkland', 'Lynnwood', 'Marysville', 
 'Olympia', 'Port Orchard', 'Puyallup', 'Redmond', 'Renton', 'Seattle', 'Sedro Woolley', 'Spokane', 'Tacoma', 'Walla Walla', 'Yakima') group by \"time_by_day\".\"the_year\", \"product_class\".\"product_family\", \"customer\".\"state_province\", \"customer\".\"city\"",
     "c0=1997; c1=Drink; c2=WA; c3=Sedro Woolley; m0=58.0000\n",
     "select \"store\".\"store_country\" as \"c0\",\n"
-      + " \"time_by_day\".\"the_year\" as \"c1\",\n"
-      + " sum(\"sales_fact_1997\".\"store_cost\") as \"m0\",\n"
-      + " count(\"sales_fact_1997\".\"product_id\") as \"m1\",\n"
-      + " count(distinct \"sales_fact_1997\".\"customer_id\") as \"m2\",\n"
-      + " sum((case when \"sales_fact_1997\".\"promotion_id\" = 0 then 0\n"
-      + "     else \"sales_fact_1997\".\"store_sales\" end)) as \"m3\"\n"
-      + "from \"store\" as \"store\",\n"
-      + " \"sales_fact_1997\" as \"sales_fact_1997\",\n"
-      + " \"time_by_day\" as \"time_by_day\"\n"
-      + "where \"sales_fact_1997\".\"store_id\" = \"store\".\"store_id\"\n"
-      + "and \"sales_fact_1997\".\"time_id\" = \"time_by_day\".\"time_id\"\n"
-      + "and \"time_by_day\".\"the_year\" = 1997\n"
-      + "group by \"store\".\"store_country\", \"time_by_day\".\"the_year\"",
+        + " \"time_by_day\".\"the_year\" as \"c1\",\n"
+        + " sum(\"sales_fact_1997\".\"store_cost\") as \"m0\",\n"
+        + " count(\"sales_fact_1997\".\"product_id\") as \"m1\",\n"
+        + " count(distinct \"sales_fact_1997\".\"customer_id\") as \"m2\",\n"
+        + " sum((case when \"sales_fact_1997\".\"promotion_id\" = 0 then 0\n"
+        + "     else \"sales_fact_1997\".\"store_sales\" end)) as \"m3\"\n"
+        + "from \"store\" as \"store\",\n"
+        + " \"sales_fact_1997\" as \"sales_fact_1997\",\n"
+        + " \"time_by_day\" as \"time_by_day\"\n"
+        + "where \"sales_fact_1997\".\"store_id\" = \"store\".\"store_id\"\n"
+        + "and \"sales_fact_1997\".\"time_id\" = \"time_by_day\".\"time_id\"\n"
+        + "and \"time_by_day\".\"the_year\" = 1997\n"
+        + "group by \"store\".\"store_country\", \"time_by_day\".\"the_year\"",
     "c0=USA; c1=1997; m0=225627.2336; m1=86837; m2=5581; m3=151211.2100\n",
       // query 6077
       // disabled (runs out of memory)
     "ignore:select \"time_by_day\".\"the_year\" as \"c0\",\n"
-      + " count(distinct \"sales_fact_1997\".\"customer_id\") as \"m0\"\n"
-      + "from \"time_by_day\" as \"time_by_day\",\n"
-      + " \"sales_fact_1997\" as \"sales_fact_1997\",\n"
-      + " \"product_class\" as \"product_class\",\n"
-      + " \"product\" as \"product\"\n"
-      + "where \"sales_fact_1997\".\"time_id\" = \"time_by_day\".\"time_id\"\n"
-      + "and \"time_by_day\".\"the_year\" = 1997\n"
-      + "and \"sales_fact_1997\".\"product_id\" = \"product\".\"product_id\"\n"
-      + "and \"product\".\"product_class_id\" = \"product_class\".\"product_class_id\"\n"
-      + "and (((\"product\".\"brand_name\" = 'Cormorant'\n"
-      + "   and \"product_class\".\"product_subcategory\" = 'Pot Scrubbers'\n"
-      + "   and \"product_class\".\"product_category\" = 'Kitchen Products'\n"
-      + "   and \"product_class\".\"product_department\" = 'Household'\n"
-      + "   and \"product_class\".\"product_family\" = 'Non-Consumable')\n"
-      + " or (\"product\".\"brand_name\" = 'Denny'\n"
-      + "   and \"product_class\".\"product_subcategory\" = 'Pot Scrubbers'\n"
-      + "   and \"product_class\".\"product_category\" = 'Kitchen Products'\n"
-      + "   and \"product_class\".\"product_department\" = 'Household'\n"
-      + "   and \"product_class\".\"product_family\" = 'Non-Consumable')\n"
-      + " or (\"product\".\"brand_name\" = 'High Quality'\n"
-      + "   and \"product_class\".\"product_subcategory\" = 'Pot Scrubbers'\n"
-      + "   and \"product_class\".\"product_category\" = 'Kitchen Products'\n"
-      + "   and \"product_class\".\"product_department\" = 'Household'\n"
-      + "   and \"product_class\".\"product_family\" = 'Non-Consumable')\n"
-      + " or (\"product\".\"brand_name\" = 'Red Wing'\n"
-      + "   and \"product_class\".\"product_subcategory\" = 'Pot Scrubbers'\n"
-      + "   and \"product_class\".\"product_category\" = 'Kitchen Products'\n"
-      + "   and \"product_class\".\"product_department\" = 'Household'\n"
-      + "   and \"product_class\".\"product_family\" = 'Non-Consumable'))\n"
-      + " or (\"product_class\".\"product_subcategory\" = 'Pots and Pans'\n"
-      + "   and \"product_class\".\"product_category\" = 'Kitchen Products'\n"
-      + "   and \"product_class\".\"product_department\" = 'Household'\n"
-      + "   and \"product_class\".\"product_family\" = 'Non-Consumable'))\n"
-      + "group by \"time_by_day\".\"the_year\"\n",
+        + " count(distinct \"sales_fact_1997\".\"customer_id\") as \"m0\"\n"
+        + "from \"time_by_day\" as \"time_by_day\",\n"
+        + " \"sales_fact_1997\" as \"sales_fact_1997\",\n"
+        + " \"product_class\" as \"product_class\",\n"
+        + " \"product\" as \"product\"\n"
+        + "where \"sales_fact_1997\".\"time_id\" = \"time_by_day\".\"time_id\"\n"
+        + "and \"time_by_day\".\"the_year\" = 1997\n"
+        + "and \"sales_fact_1997\".\"product_id\" = \"product\".\"product_id\"\n"
+        + "and \"product\".\"product_class_id\" = \"product_class\".\"product_class_id\"\n"
+        + "and (((\"product\".\"brand_name\" = 'Cormorant'\n"
+        + "   and \"product_class\".\"product_subcategory\" = 'Pot Scrubbers'\n"
+        + "   and \"product_class\".\"product_category\" = 'Kitchen Products'\n"
+        + "   and \"product_class\".\"product_department\" = 'Household'\n"
+        + "   and \"product_class\".\"product_family\" = 'Non-Consumable')\n"
+        + " or (\"product\".\"brand_name\" = 'Denny'\n"
+        + "   and \"product_class\".\"product_subcategory\" = 'Pot Scrubbers'\n"
+        + "   and \"product_class\".\"product_category\" = 'Kitchen Products'\n"
+        + "   and \"product_class\".\"product_department\" = 'Household'\n"
+        + "   and \"product_class\".\"product_family\" = 'Non-Consumable')\n"
+        + " or (\"product\".\"brand_name\" = 'High Quality'\n"
+        + "   and \"product_class\".\"product_subcategory\" = 'Pot Scrubbers'\n"
+        + "   and \"product_class\".\"product_category\" = 'Kitchen Products'\n"
+        + "   and \"product_class\".\"product_department\" = 'Household'\n"
+        + "   and \"product_class\".\"product_family\" = 'Non-Consumable')\n"
+        + " or (\"product\".\"brand_name\" = 'Red Wing'\n"
+        + "   and \"product_class\".\"product_subcategory\" = 'Pot Scrubbers'\n"
+        + "   and \"product_class\".\"product_category\" = 'Kitchen Products'\n"
+        + "   and \"product_class\".\"product_department\" = 'Household'\n"
+        + "   and \"product_class\".\"product_family\" = 'Non-Consumable'))\n"
+        + " or (\"product_class\".\"product_subcategory\" = 'Pots and Pans'\n"
+        + "   and \"product_class\".\"product_category\" = 'Kitchen Products'\n"
+        + "   and \"product_class\".\"product_department\" = 'Household'\n"
+        + "   and \"product_class\".\"product_family\" = 'Non-Consumable'))\n"
+        + "group by \"time_by_day\".\"the_year\"\n",
     "xxtodo",
       // query 6077, simplified
       // disabled (slow)
     "ignore:select count(\"sales_fact_1997\".\"customer_id\") as \"m0\"\n"
-      + "from \"sales_fact_1997\" as \"sales_fact_1997\",\n"
-      + " \"product_class\" as \"product_class\",\n"
-      + " \"product\" as \"product\"\n"
-      + "where \"sales_fact_1997\".\"product_id\" = \"product\".\"product_id\"\n"
-      + "and \"product\".\"product_class_id\" = \"product_class\".\"product_class_id\"\n"
-      + "and ((\"product\".\"brand_name\" = 'Cormorant'\n"
-      + "   and \"product_class\".\"product_subcategory\" = 'Pot Scrubbers')\n"
-      + " or (\"product_class\".\"product_subcategory\" = 'Pots and Pans'))\n",
+        + "from \"sales_fact_1997\" as \"sales_fact_1997\",\n"
+        + " \"product_class\" as \"product_class\",\n"
+        + " \"product\" as \"product\"\n"
+        + "where \"sales_fact_1997\".\"product_id\" = \"product\".\"product_id\"\n"
+        + "and \"product\".\"product_class_id\" = \"product_class\".\"product_class_id\"\n"
+        + "and ((\"product\".\"brand_name\" = 'Cormorant'\n"
+        + "   and \"product_class\".\"product_subcategory\" = 'Pot Scrubbers')\n"
+        + " or (\"product_class\".\"product_subcategory\" = 'Pots and Pans'))\n",
     "xxxx",
       // query 6077, simplified further
     "select count(distinct \"sales_fact_1997\".\"customer_id\") as \"m0\"\n"
-      + "from \"sales_fact_1997\" as \"sales_fact_1997\",\n"
-      + " \"product_class\" as \"product_class\",\n"
-      + " \"product\" as \"product\"\n"
-      + "where \"sales_fact_1997\".\"product_id\" = \"product\".\"product_id\"\n"
-      + "and \"product\".\"product_class_id\" = \"product_class\".\"product_class_id\"\n"
-      + "and \"product\".\"brand_name\" = 'Cormorant'\n",
+        + "from \"sales_fact_1997\" as \"sales_fact_1997\",\n"
+        + " \"product_class\" as \"product_class\",\n"
+        + " \"product\" as \"product\"\n"
+        + "where \"sales_fact_1997\".\"product_id\" = \"product\".\"product_id\"\n"
+        + "and \"product\".\"product_class_id\" = \"product_class\".\"product_class_id\"\n"
+        + "and \"product\".\"brand_name\" = 'Cormorant'\n",
     "m0=1298",
       // query 193
     "select \"store\".\"store_country\" as \"c0\",\n"
-      + " \"time_by_day\".\"the_year\" as \"c1\",\n"
-      + " \"time_by_day\".\"quarter\" as \"c2\",\n"
-      + " \"product_class\".\"product_family\" as \"c3\",\n"
-      + " count(\"sales_fact_1997\".\"product_id\") as \"m0\",\n"
-      + " count(distinct \"sales_fact_1997\".\"customer_id\") as \"m1\"\n"
-      + "from \"store\" as \"store\",\n"
-      + " \"sales_fact_1997\" as \"sales_fact_1997\",\n"
-      + " \"time_by_day\" as \"time_by_day\",\n"
-      + " \"product_class\" as \"product_class\",\n"
-      + " \"product\" as \"product\"\n"
-      + "where \"sales_fact_1997\".\"store_id\" = \"store\".\"store_id\"\n"
-      + "and \"store\".\"store_country\" = 'USA'\n"
-      + "and \"sales_fact_1997\".\"time_id\" = \"time_by_day\".\"time_id\"\n"
-      + "and \"time_by_day\".\"the_year\" = 1997\n"
-      + "and \"time_by_day\".\"quarter\" = 'Q3'\n"
-      + "and \"sales_fact_1997\".\"product_id\" = \"product\".\"product_id\"\n"
-      + "and \"product\".\"product_class_id\" = \"product_class\".\"product_class_id\"\n"
-      + "and \"product_class\".\"product_family\" = 'Food'\n"
-      + "group by \"store\".\"store_country\",\n"
-      + " \"time_by_day\".\"the_year\",\n"
-      + " \"time_by_day\".\"quarter\",\n"
-      + " \"product_class\".\"product_family\"",
+        + " \"time_by_day\".\"the_year\" as \"c1\",\n"
+        + " \"time_by_day\".\"quarter\" as \"c2\",\n"
+        + " \"product_class\".\"product_family\" as \"c3\",\n"
+        + " count(\"sales_fact_1997\".\"product_id\") as \"m0\",\n"
+        + " count(distinct \"sales_fact_1997\".\"customer_id\") as \"m1\"\n"
+        + "from \"store\" as \"store\",\n"
+        + " \"sales_fact_1997\" as \"sales_fact_1997\",\n"
+        + " \"time_by_day\" as \"time_by_day\",\n"
+        + " \"product_class\" as \"product_class\",\n"
+        + " \"product\" as \"product\"\n"
+        + "where \"sales_fact_1997\".\"store_id\" = \"store\".\"store_id\"\n"
+        + "and \"store\".\"store_country\" = 'USA'\n"
+        + "and \"sales_fact_1997\".\"time_id\" = \"time_by_day\".\"time_id\"\n"
+        + "and \"time_by_day\".\"the_year\" = 1997\n"
+        + "and \"time_by_day\".\"quarter\" = 'Q3'\n"
+        + "and \"sales_fact_1997\".\"product_id\" = \"product\".\"product_id\"\n"
+        + "and \"product\".\"product_class_id\" = \"product_class\".\"product_class_id\"\n"
+        + "and \"product_class\".\"product_family\" = 'Food'\n"
+        + "group by \"store\".\"store_country\",\n"
+        + " \"time_by_day\".\"the_year\",\n"
+        + " \"time_by_day\".\"quarter\",\n"
+        + " \"product_class\".\"product_family\"",
     "c0=USA; c1=1997; c2=Q3; c3=Food; m0=15449; m1=2939",
   };
 
@@ -1387,8 +1411,8 @@ public class JdbcTest {
    * running queries against the JDBC adapter. As of janino-2.7.3 bug is
    * open but we have a workaround in EnumerableRelImplementor. */
   @Test public void testJanino169() {
-    OptiqAssert.that()
-        .with(OptiqAssert.Config.JDBC_FOODMART)
+    CalciteAssert.that()
+        .with(CalciteAssert.Config.JDBC_FOODMART)
         .query(
             "select \"time_id\" from \"foodmart\".\"time_by_day\" as \"t\"\n")
         .returnsCount(730);
@@ -1401,22 +1425,20 @@ public class JdbcTest {
    * is ignored and rows with deptno=10 are wrongly returned.</p>
    */
   @Test public void testAnd3() {
-    OptiqAssert.that()
-        .with(OptiqAssert.Config.REGULAR)
-        .query(
-            "select \"deptno\" from \"hr\".\"emps\"\n"
+    CalciteAssert.that()
+        .with(CalciteAssert.Config.REGULAR)
+        .query("select \"deptno\" from \"hr\".\"emps\"\n"
             + "where \"emps\".\"empid\" < 240\n"
-                + "and \"salary\" > 7500.0"
-                + "and \"emps\".\"deptno\" > 10\n")
+            + "and \"salary\" > 7500.0"
+            + "and \"emps\".\"deptno\" > 10\n")
         .returnsUnordered("deptno=20");
   }
 
   /** Tests a date literal against a JDBC data source. */
   @Test public void testJdbcDate() {
-    OptiqAssert.that()
-        .with(OptiqAssert.Config.FOODMART_CLONE)
-        .query(
-            "select count(*) as c from (\n"
+    CalciteAssert.that()
+        .with(CalciteAssert.Config.FOODMART_CLONE)
+        .query("select count(*) as c from (\n"
             + "  select 1 from \"foodmart\".\"employee\" as e1\n"
             + "  where \"position_title\" = 'VP Country Manager'\n"
             + "  and \"birth_date\" < DATE '1950-01-01'\n"
@@ -1426,13 +1448,12 @@ public class JdbcTest {
 
   /** Tests a timestamp literal against JDBC data source. */
   @Test public void testJdbcTimestamp() {
-    OptiqAssert.that()
-        .with(OptiqAssert.Config.JDBC_FOODMART)
-        .query(
-            "select count(*) as c from (\n"
-                + "  select 1 from \"foodmart\".\"employee\" as e1\n"
-                + "  where \"hire_date\" < TIMESTAMP '1996-06-05 00:00:00'\n"
-                + "  and \"gender\" = 'F')")
+    CalciteAssert.that()
+        .with(CalciteAssert.Config.JDBC_FOODMART)
+        .query("select count(*) as c from (\n"
+            + "  select 1 from \"foodmart\".\"employee\" as e1\n"
+            + "  where \"hire_date\" < TIMESTAMP '1996-06-05 00:00:00'\n"
+            + "  and \"gender\" = 'F')")
         .returns("C=287\n");
   }
 
@@ -1440,10 +1461,9 @@ public class JdbcTest {
    * <a href="https://issues.apache.org/jira/browse/CALCITE-281">CALCITE-281</a>,
    * "SQL type of EXTRACT is BIGINT but it is implemented as int". */
   @Test public void testExtract() {
-    OptiqAssert.that()
-        .with(OptiqAssert.Config.JDBC_FOODMART)
-        .query(
-            "values extract(year from date '2008-2-23')")
+    CalciteAssert.that()
+        .with(CalciteAssert.Config.JDBC_FOODMART)
+        .query("values extract(year from date '2008-2-23')")
         .returns(
             new Function<ResultSet, Void>() {
               public Void apply(ResultSet a0) {
@@ -1462,17 +1482,14 @@ public class JdbcTest {
    * <a href="https://issues.apache.org/jira/browse/CALCITE-387">CALCITE-387</a>,
    * "CompileException when cast TRUE to nullable boolean". */
   @Test public void testTrue() {
-    final OptiqAssert.AssertThat that = OptiqAssert.that();
-    that.query(
-        "select case when deptno = 10 then null else true end as x\n"
+    final CalciteAssert.AssertThat that = CalciteAssert.that();
+    that.query("select case when deptno = 10 then null else true end as x\n"
         + "from (values (10), (20)) as t(deptno)")
         .returnsUnordered("X=null", "X=true");
-    that.query(
-        "select case when deptno = 10 then null else 100 end as x\n"
+    that.query("select case when deptno = 10 then null else 100 end as x\n"
         + "from (values (10), (20)) as t(deptno)")
         .returnsUnordered("X=null", "X=100");
-    that.query(
-        "select case when deptno = 10 then null else 'xy' end as x\n"
+    that.query("select case when deptno = 10 then null else 'xy' end as x\n"
         + "from (values (10), (20)) as t(deptno)")
         .returnsUnordered("X=null", "X=xy");
   }
@@ -1480,10 +1497,9 @@ public class JdbcTest {
   /** Unit test for self-join. Left and right children of the join are the same
    * relational expression. */
   @Test public void testSelfJoin() {
-    OptiqAssert.that()
-        .with(OptiqAssert.Config.JDBC_FOODMART)
-        .query(
-            "select count(*) as c from (\n"
+    CalciteAssert.that()
+        .with(CalciteAssert.Config.JDBC_FOODMART)
+        .query("select count(*) as c from (\n"
             + "  select 1 from \"foodmart\".\"employee\" as e1\n"
             + "  join \"foodmart\".\"employee\" as e2 using (\"position_title\"))")
         .returns("C=247149\n");
@@ -1492,15 +1508,13 @@ public class JdbcTest {
   /** Self-join on different columns, select a different column, and sort and
    * limit on yet another column. */
   @Test public void testSelfJoinDifferentColumns() {
-    OptiqAssert.that()
-        .with(OptiqAssert.Config.JDBC_FOODMART)
-        .query(
-            "select e1.\"full_name\"\n"
+    CalciteAssert.that()
+        .with(CalciteAssert.Config.JDBC_FOODMART)
+        .query("select e1.\"full_name\"\n"
             + "  from \"foodmart\".\"employee\" as e1\n"
             + "  join \"foodmart\".\"employee\" as e2 on e1.\"first_name\" = e2.\"last_name\"\n"
             + "order by e1.\"last_name\" limit 3")
-        .returns(
-            "full_name=James Aguilar\n"
+        .returns("full_name=James Aguilar\n"
             + "full_name=Carol Amyotte\n"
             + "full_name=Terry Anderson\n");
   }
@@ -1511,16 +1525,14 @@ public class JdbcTest {
    * <a href="https://issues.apache.org/jira/browse/CALCITE-371">CALCITE-371</a>,
    * "Cannot implement JOIN whose ON clause contains mixed equi and theta". */
   @Test public void testEquiThetaJoin() {
-    OptiqAssert.that()
-        .with(OptiqAssert.Config.REGULAR)
-        .query(
-            "select e.\"empid\", d.\"name\", e.\"name\"\n"
+    CalciteAssert.that()
+        .with(CalciteAssert.Config.REGULAR)
+        .query("select e.\"empid\", d.\"name\", e.\"name\"\n"
             + "from \"hr\".\"emps\" as e\n"
             + "join \"hr\".\"depts\" as d\n"
             + "on e.\"deptno\" = d.\"deptno\"\n"
             + "and e.\"name\" <> d.\"name\"\n")
-        .returns(
-            "empid=100; name=Sales; name=Bill\n"
+        .returns("empid=100; name=Sales; name=Bill\n"
             + "empid=150; name=Sales; name=Sebastian\n"
             + "empid=110; name=Sales; name=Theodore\n");
   }
@@ -1530,10 +1542,9 @@ public class JdbcTest {
    * "Support parenthesized sub-clause in JOIN". */
   @Ignore
   @Test public void testJoinJoin() {
-    OptiqAssert.that()
-        .with(OptiqAssert.Config.FOODMART_CLONE)
-        .query(
-            "select\n"
+    CalciteAssert.that()
+        .with(CalciteAssert.Config.FOODMART_CLONE)
+        .query("select\n"
             + "   \"product_class\".\"product_family\" as \"c0\",\n"
             + "   \"product_class\".\"product_department\" as \"c1\",\n"
             + "   \"customer\".\"country\" as \"c2\",\n"
@@ -1566,8 +1577,7 @@ public class JdbcTest {
             + "   ISNULL(\"customer\".\"country\") ASC,   \"customer\".\"country\" ASC,\n"
             + "   ISNULL(\"customer\".\"state_province\") ASC,   \"customer\".\"state_province\" ASC,\n"
             + "   ISNULL(\"customer\".\"city\") ASC,   \"customer\".\"city\" ASC")
-        .returns(
-            "+-------+---------------------+-----+------+------------+\n"
+        .returns("+-------+---------------------+-----+------+------------+\n"
             + "| c0    | c1                  | c2  | c3   | c4         |\n"
             + "+-------+---------------------+-----+------+------------+\n"
             + "| Drink | Alcoholic Beverages | USA | WA   | Bellingham |\n"
@@ -1578,10 +1588,9 @@ public class JdbcTest {
   /** Four-way join. Used to take 80 seconds. */
   @Ignore
   @Test public void testJoinFiveWay() {
-    OptiqAssert.that()
-        .with(OptiqAssert.Config.FOODMART_CLONE)
-        .query(
-            "select \"store\".\"store_country\" as \"c0\",\n"
+    CalciteAssert.that()
+        .with(CalciteAssert.Config.FOODMART_CLONE)
+        .query("select \"store\".\"store_country\" as \"c0\",\n"
             + " \"time_by_day\".\"the_year\" as \"c1\",\n"
             + " \"product_class\".\"product_family\" as \"c2\",\n"
             + " count(\"sales_fact_1997\".\"product_id\") as \"m0\"\n"
@@ -1599,26 +1608,25 @@ public class JdbcTest {
             + "group by \"store\".\"store_country\",\n"
             + " \"time_by_day\".\"the_year\",\n"
             + " \"product_class\".\"product_family\"")
-        .explainContains(
-            "EnumerableAggregateRel(group=[{0, 1, 2}], m0=[COUNT($3)])\n"
+        .explainContains(""
+            + "EnumerableAggregateRel(group=[{0, 1, 2}], m0=[COUNT($3)])\n"
             + "  EnumerableCalcRel(expr#0..61=[{inputs}], c0=[$t19], c1=[$t4], c2=[$t46], product_id=[$t34])\n"
             + "    EnumerableJoinRel(condition=[=($35, $0)], joinType=[inner])\n"
             + "      EnumerableCalcRel(expr#0..9=[{inputs}], expr#10=[CAST($t4):INTEGER], expr#11=[1997], expr#12=[=($t10, $t11)], proj#0..9=[{exprs}], $condition=[$t12])\n"
-            + "        EnumerableTableAccessRel(table=[[foodmart2, time_by_day]])\n"
+            + "        EnumerableTableScan(table=[[foodmart2, time_by_day]])\n"
             + "      EnumerableCalcRel(expr#0..51=[{inputs}], proj#0..23=[{exprs}], product_id=[$t44], time_id=[$t45], customer_id=[$t46], promotion_id=[$t47], store_id0=[$t48], store_sales=[$t49], store_cost=[$t50], unit_sales=[$t51], product_class_id=[$t24], product_subcategory=[$t25], product_category=[$t26], product_department=[$t27], product_family=[$t28], product_class_id0=[$t29], product_id0=[$t30], brand_name=[$t31], product_name=[$t32], SKU=[$t33], SRP=[$t34], gross_weight=[$t35], net_weight=[$t36], recyclable_package=[$t37], low_fat=[$t38], units_per_case=[$t39], cases_per_pallet=[$t40], shelf_width=[$t41], shelf_height=[$t42], shelf_depth=[$t43])\n"
             + "        EnumerableJoinRel(condition=[=($48, $0)], joinType=[inner])\n"
             + "          EnumerableCalcRel(expr#0..23=[{inputs}], expr#24=['USA'], expr#25=[=($t9, $t24)], proj#0..23=[{exprs}], $condition=[$t25])\n"
-            + "            EnumerableTableAccessRel(table=[[foodmart2, store]])\n"
+            + "            EnumerableTableScan(table=[[foodmart2, store]])\n"
             + "          EnumerableCalcRel(expr#0..27=[{inputs}], proj#0..4=[{exprs}], product_class_id0=[$t13], product_id=[$t14], brand_name=[$t15], product_name=[$t16], SKU=[$t17], SRP=[$t18], gross_weight=[$t19], net_weight=[$t20], recyclable_package=[$t21], low_fat=[$t22], units_per_case=[$t23], cases_per_pallet=[$t24], shelf_width=[$t25], shelf_height=[$t26], shelf_depth=[$t27], product_id0=[$t5], time_id=[$t6], customer_id=[$t7], promotion_id=[$t8], store_id=[$t9], store_sales=[$t10], store_cost=[$t11], unit_sales=[$t12])\n"
             + "            EnumerableJoinRel(condition=[=($13, $0)], joinType=[inner])\n"
-            + "              EnumerableTableAccessRel(table=[[foodmart2, product_class]])\n"
+            + "              EnumerableTableScan(table=[[foodmart2, product_class]])\n"
             + "              EnumerableJoinRel(condition=[=($0, $9)], joinType=[inner])\n"
-            + "                EnumerableTableAccessRel(table=[[foodmart2, sales_fact_1997]])\n"
-            + "                EnumerableTableAccessRel(table=[[foodmart2, product]])\n"
+            + "                EnumerableTableScan(table=[[foodmart2, sales_fact_1997]])\n"
+            + "                EnumerableTableScan(table=[[foodmart2, product]])\n"
             + "\n"
             + "]>")
-        .returns(
-            "+-------+---------------------+-----+------+------------+\n"
+        .returns("+-------+---------------------+-----+------+------------+\n"
             + "| c0    | c1                  | c2  | c3   | c4         |\n"
             + "+-------+---------------------+-----+------+------------+\n"
             + "| Drink | Alcoholic Beverages | USA | WA   | Bellingham |\n"
@@ -1654,8 +1662,8 @@ public class JdbcTest {
           .append(i).append(".\"deptno\" = d")
           .append(i - 1).append(".\"deptno\"");
     }
-    OptiqAssert.that()
-        .with(OptiqAssert.Config.REGULAR)
+    CalciteAssert.that()
+        .with(CalciteAssert.Config.REGULAR)
         .query(buf.toString())
         .returns("EXPR$0=3\n");
   }
@@ -1681,9 +1689,9 @@ public class JdbcTest {
   /** A selection of queries generated by Mondrian. */
   @Ignore
   @Test public void testCloneQueries() {
-    OptiqAssert.AssertThat with =
-        OptiqAssert.that()
-            .with(OptiqAssert.Config.FOODMART_CLONE);
+    CalciteAssert.AssertThat with =
+        CalciteAssert.that()
+            .with(CalciteAssert.Config.FOODMART_CLONE);
     for (Ord<Pair<String, String>> query : Ord.zip(FOODMART_QUERIES)) {
       try {
         // uncomment to run specific queries:
@@ -1693,7 +1701,7 @@ public class JdbcTest {
           continue;
         }
         final String expected = query.e.right;
-        final OptiqAssert.AssertQuery query1 = with.query(sql);
+        final CalciteAssert.AssertQuery query1 = with.query(sql);
         if (expected != null) {
           if (sql.contains("order by")) {
             query1.returns(expected);
@@ -1714,13 +1722,11 @@ public class JdbcTest {
     String hsqldbMemUrl = "jdbc:hsqldb:mem:.";
     Connection baseConnection = DriverManager.getConnection(hsqldbMemUrl);
     Statement baseStmt = baseConnection.createStatement();
-    baseStmt.execute(
-        "CREATE TABLE ARR_TABLE (\n"
+    baseStmt.execute("CREATE TABLE ARR_TABLE (\n"
         + "ID INTEGER,\n"
         + "VALS INTEGER ARRAY)");
     baseStmt.execute("INSERT INTO ARR_TABLE VALUES (1, ARRAY[1,2,3])");
-    baseStmt.execute(
-        "CREATE TABLE ARR_TABLE2 (\n"
+    baseStmt.execute("CREATE TABLE ARR_TABLE2 (\n"
         + "ID INTEGER,\n"
         + "VALS INTEGER ARRAY,\n"
         + "VALVALS VARCHAR(10) ARRAY)");
@@ -1731,37 +1737,37 @@ public class JdbcTest {
 
     Properties info = new Properties();
     info.put("model",
-      "inline:"
-        + "{\n"
-        + "  version: '1.0',\n"
-        + "  defaultSchema: 'BASEJDBC',\n"
-        + "  schemas: [\n"
-        + "     {\n"
-        + "       type: 'jdbc',\n"
-        + "       name: 'BASEJDBC',\n"
-        + "       jdbcDriver: '" + jdbcDriver.class.getName() + "',\n"
-        + "       jdbcUrl: '" + hsqldbMemUrl + "',\n"
-        + "       jdbcCatalog: null,\n"
-        + "       jdbcSchema: null\n"
-        + "     }\n"
-        + "  ]\n"
-        + "}");
+        "inline:"
+            + "{\n"
+            + "  version: '1.0',\n"
+            + "  defaultSchema: 'BASEJDBC',\n"
+            + "  schemas: [\n"
+            + "     {\n"
+            + "       type: 'jdbc',\n"
+            + "       name: 'BASEJDBC',\n"
+            + "       jdbcDriver: '" + jdbcDriver.class.getName() + "',\n"
+            + "       jdbcUrl: '" + hsqldbMemUrl + "',\n"
+            + "       jdbcCatalog: null,\n"
+            + "       jdbcSchema: null\n"
+            + "     }\n"
+            + "  ]\n"
+            + "}");
 
-    Connection optiqConnection = DriverManager.getConnection(
+    Connection calciteConnection = DriverManager.getConnection(
       "jdbc:calcite:", info);
 
-    Statement optiqStatement = optiqConnection.createStatement();
-    ResultSet rs = optiqStatement.executeQuery(
+    Statement calciteStatement = calciteConnection.createStatement();
+    ResultSet rs = calciteStatement.executeQue

<TRUNCATED>

[52/58] [abbrv] incubator-calcite git commit: [CALCITE-460] Add ImmutableBitSet and replace uses of BitSet

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/core/src/test/java/org/apache/calcite/util/ImmutableBitSetTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/util/ImmutableBitSetTest.java b/core/src/test/java/org/apache/calcite/util/ImmutableBitSetTest.java
new file mode 100644
index 0000000..84dc9e4
--- /dev/null
+++ b/core/src/test/java/org/apache/calcite/util/ImmutableBitSetTest.java
@@ -0,0 +1,383 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.util;
+
+import org.apache.calcite.runtime.Utilities;
+
+import com.google.common.collect.Maps;
+
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.SortedMap;
+
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * Unit test for {@link org.apache.calcite.util.ImmutableBitSet}.
+ */
+public class ImmutableBitSetTest {
+  /** Tests the method {@link ImmutableBitSet#iterator()}. */
+  @Test public void testIterator() {
+    assertToIterBitSet("", ImmutableBitSet.of());
+    assertToIterBitSet("0", ImmutableBitSet.of(0));
+    assertToIterBitSet("0, 1", ImmutableBitSet.of(0, 1));
+    assertToIterBitSet("10", ImmutableBitSet.of(10));
+  }
+
+  /**
+   * Tests that iterating over an
+   * {@link org.apache.calcite.util.ImmutableBitSet} yields the expected string.
+   *
+   * @param expected Expected string
+   * @param bitSet   Bit set
+   */
+  private void assertToIterBitSet(String expected, ImmutableBitSet bitSet) {
+    StringBuilder buf = new StringBuilder();
+    for (int i : bitSet) {
+      if (buf.length() > 0) {
+        buf.append(", ");
+      }
+      buf.append(Integer.toString(i));
+    }
+    assertEquals(expected, buf.toString());
+  }
+
+  /**
+   * Tests the method
+   * {@link org.apache.calcite.util.ImmutableBitSet#toList()}.
+   */
+  @Test public void testToList() {
+    assertThat(ImmutableBitSet.of().toList(),
+        equalTo(Collections.<Integer>emptyList()));
+    assertThat(ImmutableBitSet.of(5).toList(), equalTo(Arrays.asList(5)));
+    assertThat(ImmutableBitSet.of(3, 5).toList(), equalTo(Arrays.asList(3, 5)));
+    assertThat(ImmutableBitSet.of(63).toList(), equalTo(Arrays.asList(63)));
+    assertThat(ImmutableBitSet.of(64).toList(), equalTo(Arrays.asList(64)));
+    assertThat(ImmutableBitSet.of(3, 63).toList(),
+        equalTo(Arrays.asList(3, 63)));
+    assertThat(ImmutableBitSet.of(3, 64).toList(),
+        equalTo(Arrays.asList(3, 64)));
+    assertThat(ImmutableBitSet.of(0, 4, 2).toList(),
+        equalTo(Arrays.asList(0, 2, 4)));
+  }
+
+  /**
+   * Tests the method {@link BitSets#range(int, int)}.
+   */
+  @Test public void testRange() {
+    assertEquals(ImmutableBitSet.range(0, 4).toList(),
+        Arrays.asList(0, 1, 2, 3));
+    assertEquals(ImmutableBitSet.range(1, 4).toList(),
+        Arrays.asList(1, 2, 3));
+    assertEquals(ImmutableBitSet.range(4).toList(),
+        Arrays.asList(0, 1, 2, 3));
+    assertEquals(ImmutableBitSet.range(0).toList(),
+        Collections.<Integer>emptyList());
+    assertEquals(ImmutableBitSet.range(2, 2).toList(),
+        Collections.<Integer>emptyList());
+
+    assertThat(ImmutableBitSet.range(63, 66).toString(),
+        equalTo("{63, 64, 65}"));
+    assertThat(ImmutableBitSet.range(65, 68).toString(),
+        equalTo("{65, 66, 67}"));
+    assertThat(ImmutableBitSet.range(65, 65).toString(), equalTo("{}"));
+    assertThat(ImmutableBitSet.range(65, 65).length(), equalTo(0));
+    assertThat(ImmutableBitSet.range(65, 165).cardinality(), equalTo(100));
+
+    // Same tests as above, using a builder.
+    assertThat(ImmutableBitSet.builder().set(63, 66).build().toString(),
+        equalTo("{63, 64, 65}"));
+    assertThat(ImmutableBitSet.builder().set(65, 68).build().toString(),
+        equalTo("{65, 66, 67}"));
+    assertThat(ImmutableBitSet.builder().set(65, 65).build().toString(),
+        equalTo("{}"));
+    assertThat(ImmutableBitSet.builder().set(65, 65).build().length(),
+        equalTo(0));
+    assertThat(ImmutableBitSet.builder().set(65, 165).build().cardinality(),
+        equalTo(100));
+
+    final ImmutableBitSet e0 = ImmutableBitSet.range(0, 0);
+    final ImmutableBitSet e1 = ImmutableBitSet.of();
+    assertTrue(e0.equals(e1));
+    assertThat(e0.hashCode(), equalTo(e1.hashCode()));
+
+    // Empty builder returns the singleton empty set.
+    assertTrue(ImmutableBitSet.builder().build() == ImmutableBitSet.of());
+  }
+
+  @Test public void testCompare() {
+    final List<ImmutableBitSet> sorted = getSortedList();
+    for (int i = 0; i < sorted.size(); i++) {
+      for (int j = 0; j < sorted.size(); j++) {
+        final ImmutableBitSet set0 = sorted.get(i);
+        final ImmutableBitSet set1 = sorted.get(j);
+        int c = set0.compareTo(set1);
+        if (c == 0) {
+          assertTrue(i == j || i == 3 && j == 4 || i == 4 && j == 3);
+        } else {
+          assertEquals(c, Utilities.compare(i, j));
+        }
+        assertEquals(c == 0, set0.equals(set1));
+        assertEquals(c == 0, set1.equals(set0));
+      }
+    }
+  }
+
+  @Test public void testCompare2() {
+    final List<ImmutableBitSet> sorted = getSortedList();
+    Collections.sort(sorted, ImmutableBitSet.COMPARATOR);
+    assertThat(sorted.toString(),
+        equalTo("[{0, 1, 3}, {0, 1}, {1, 1000}, {1}, {1}, {2, 3}, {}]"));
+  }
+
+  private List<ImmutableBitSet> getSortedList() {
+    return Arrays.asList(
+        ImmutableBitSet.of(),
+        ImmutableBitSet.of(0, 1),
+        ImmutableBitSet.of(0, 1, 3),
+        ImmutableBitSet.of(1),
+        ImmutableBitSet.of(1),
+        ImmutableBitSet.of(1, 1000),
+        ImmutableBitSet.of(2, 3));
+  }
+
+  /**
+   * Tests the method
+   * {@link org.apache.calcite.util.ImmutableBitSet#toArray}.
+   */
+  @Test public void testToArray() {
+    int[][] arrays = {{}, {0}, {0, 2}, {1, 65}, {100}};
+    for (int[] array : arrays) {
+      assertThat(ImmutableBitSet.of(array).toArray(), equalTo(array));
+    }
+  }
+
+  /**
+   * Tests the methods
+   * {@link org.apache.calcite.util.ImmutableBitSet#toList} and
+   * {@link org.apache.calcite.util.ImmutableBitSet#asList}.
+   */
+  @Test public void testAsList() {
+    final List<ImmutableBitSet> list = getSortedList();
+    for (ImmutableBitSet bitSet : list) {
+      final IntList list1 = bitSet.toList();
+      final List<Integer> listView = bitSet.asList();
+      assertThat(list1.size(), equalTo(bitSet.cardinality()));
+      assertThat(list1.toString(), equalTo(listView.toString()));
+      assertTrue(list1.equals(listView));
+      assertThat(list1.hashCode(), equalTo(listView.hashCode()));
+    }
+  }
+
+  /**
+   * Tests the method
+   * {@link org.apache.calcite.util.ImmutableBitSet#union(ImmutableBitSet)}.
+   */
+  @Test public void testUnion() {
+    assertThat(ImmutableBitSet.of(1).union(ImmutableBitSet.of(3)).toString(),
+        equalTo("{1, 3}"));
+    assertThat(ImmutableBitSet.of(1).union(ImmutableBitSet.of(3, 100))
+            .toString(),
+        equalTo("{1, 3, 100}"));
+    ImmutableBitSet x =
+        ImmutableBitSet.builder(ImmutableBitSet.of(1))
+            .addAll(ImmutableBitSet.of(2))
+            .addAll(ImmutableBitSet.of())
+            .addAll(ImmutableBitSet.of(3))
+            .build();
+    assertThat(x.toString(), equalTo("{1, 2, 3}"));
+  }
+
+  @Test public void testIntersect() {
+    assertThat(ImmutableBitSet.of(1, 2, 3, 100, 200)
+        .intersect(ImmutableBitSet.of(2, 100)).toString(), equalTo("{2, 100}"));
+    assertTrue(ImmutableBitSet.of(1, 3, 5, 101, 20001)
+        .intersect(ImmutableBitSet.of(2, 100)) == ImmutableBitSet.of());
+  }
+
+  /**
+   * Tests the method
+   * {@link org.apache.calcite.util.ImmutableBitSet#contains(org.apache.calcite.util.ImmutableBitSet)}.
+   */
+  @Test public void testBitSetsContains() {
+    assertTrue(ImmutableBitSet.range(0, 5)
+        .contains(ImmutableBitSet.range(2, 4)));
+    assertTrue(ImmutableBitSet.range(0, 5).contains(ImmutableBitSet.range(4)));
+    assertFalse(ImmutableBitSet.range(0, 5).contains(ImmutableBitSet.of(14)));
+    assertFalse(ImmutableBitSet.range(20, 25).contains(ImmutableBitSet.of(14)));
+    final ImmutableBitSet empty = ImmutableBitSet.of();
+    assertTrue(ImmutableBitSet.range(20, 25).contains(empty));
+    assertTrue(empty.contains(empty));
+    assertFalse(empty.contains(ImmutableBitSet.of(0)));
+    assertFalse(empty.contains(ImmutableBitSet.of(1)));
+    assertFalse(empty.contains(ImmutableBitSet.of(63)));
+    assertFalse(empty.contains(ImmutableBitSet.of(64)));
+    assertFalse(empty.contains(ImmutableBitSet.of(1000)));
+    assertTrue(ImmutableBitSet.of(1, 4, 7)
+        .contains(ImmutableBitSet.of(1, 4, 7)));
+  }
+
+  /**
+   * Tests the method
+   * {@link org.apache.calcite.util.ImmutableBitSet#of(org.apache.calcite.util.ImmutableIntList)}.
+   */
+  @Test public void testBitSetOfImmutableIntList() {
+    ImmutableIntList list = ImmutableIntList.of();
+    assertThat(ImmutableBitSet.of(list), equalTo(ImmutableBitSet.of()));
+
+    list = ImmutableIntList.of(2, 70, 5, 0);
+    assertThat(ImmutableBitSet.of(list),
+        equalTo(ImmutableBitSet.of(0, 2, 5, 70)));
+  }
+
+  /**
+   * Tests the method
+   * {@link org.apache.calcite.util.ImmutableBitSet#previousClearBit(int)}.
+   */
+  @Test public void testPreviousClearBit() {
+    assertThat(ImmutableBitSet.of().previousClearBit(10), equalTo(10));
+    assertThat(ImmutableBitSet.of().previousClearBit(0), equalTo(0));
+    assertThat(ImmutableBitSet.of().previousClearBit(-1), equalTo(-1));
+    try {
+      final int actual = ImmutableBitSet.of().previousClearBit(-2);
+      fail("expected exception, got " + actual);
+    } catch (IndexOutOfBoundsException e) {
+      // ok
+    }
+    assertThat(ImmutableBitSet.of(0, 1, 3, 4).previousClearBit(4), equalTo(2));
+    assertThat(ImmutableBitSet.of(0, 1, 3, 4).previousClearBit(3), equalTo(2));
+    assertThat(ImmutableBitSet.of(0, 1, 3, 4).previousClearBit(2), equalTo(2));
+    assertThat(ImmutableBitSet.of(0, 1, 3, 4).previousClearBit(1),
+        equalTo(-1));
+    assertThat(ImmutableBitSet.of(1, 3, 4).previousClearBit(1), equalTo(0));
+  }
+
+  @Test public void testBuilder() {
+    assertThat(ImmutableBitSet.builder().set(9)
+            .set(100)
+            .set(1000)
+            .clear(250)
+            .set(88)
+            .clear(100)
+            .clear(1000)
+            .build().toString(),
+        equalTo("{9, 88}"));
+  }
+
+  /** Unit test for
+   * {@link org.apache.calcite.util.ImmutableBitSet.Builder#build(ImmutableBitSet)}. */
+  @Test public void testBuilderUseOriginal() {
+    final ImmutableBitSet fives = ImmutableBitSet.of(5, 10, 15);
+    final ImmutableBitSet fives2 =
+        ImmutableBitSet.builder(fives).clear(2).set(10).build(fives);
+    assertTrue(fives2 == fives);
+    final ImmutableBitSet fives3 =
+        ImmutableBitSet.builder(fives).clear(2).set(10).build();
+    assertTrue(fives3 != fives);
+    assertTrue(fives3.equals(fives));
+    assertTrue(fives3.equals(fives2));
+  }
+
+  @Test public void testIndexOf() {
+    assertThat(ImmutableBitSet.of(0, 2, 4).indexOf(0), equalTo(0));
+    assertThat(ImmutableBitSet.of(0, 2, 4).indexOf(2), equalTo(1));
+    assertThat(ImmutableBitSet.of(0, 2, 4).indexOf(3), equalTo(-1));
+    assertThat(ImmutableBitSet.of(0, 2, 4).indexOf(4), equalTo(2));
+    assertThat(ImmutableBitSet.of(0, 2, 4).indexOf(5), equalTo(-1));
+    assertThat(ImmutableBitSet.of(0, 2, 4).indexOf(-1), equalTo(-1));
+    assertThat(ImmutableBitSet.of(0, 2, 4).indexOf(-2), equalTo(-1));
+    assertThat(ImmutableBitSet.of().indexOf(-1), equalTo(-1));
+    assertThat(ImmutableBitSet.of().indexOf(-2), equalTo(-1));
+    assertThat(ImmutableBitSet.of().indexOf(0), equalTo(-1));
+    assertThat(ImmutableBitSet.of().indexOf(1000), equalTo(-1));
+  }
+
+  @Test public void testNth() {
+    assertThat(ImmutableBitSet.of(0, 2, 4).nth(0), equalTo(0));
+    assertThat(ImmutableBitSet.of(0, 2, 4).nth(1), equalTo(2));
+    assertThat(ImmutableBitSet.of(0, 2, 4).nth(2), equalTo(4));
+    assertThat(ImmutableBitSet.of(0, 2, 63).nth(2), equalTo(63));
+    assertThat(ImmutableBitSet.of(0, 2, 64).nth(2), equalTo(64));
+    assertThat(ImmutableBitSet.of(64).nth(0), equalTo(64));
+    assertThat(ImmutableBitSet.of(64, 65).nth(0), equalTo(64));
+    assertThat(ImmutableBitSet.of(64, 65).nth(1), equalTo(65));
+    assertThat(ImmutableBitSet.of(64, 128).nth(1), equalTo(128));
+    try {
+      ImmutableBitSet.of().nth(0);
+      fail("expected throw");
+    } catch (IndexOutOfBoundsException e) {
+      // ok
+    }
+    try {
+      ImmutableBitSet.of().nth(1);
+      fail("expected throw");
+    } catch (IndexOutOfBoundsException e) {
+      // ok
+    }
+    try {
+      ImmutableBitSet.of(64).nth(1);
+      fail("expected throw");
+    } catch (IndexOutOfBoundsException e) {
+      // ok
+    }
+    try {
+      ImmutableBitSet.of(64).nth(-1);
+      fail("expected throw");
+    } catch (IndexOutOfBoundsException e) {
+      // ok
+    }
+  }
+
+  /** Tests the method
+   * {@link org.apache.calcite.util.BitSets#closure(java.util.SortedMap)}. */
+  @Test public void testClosure() {
+    final SortedMap<Integer, ImmutableBitSet> empty = Maps.newTreeMap();
+    assertThat(ImmutableBitSet.closure(empty), equalTo(empty));
+
+    // Currently you need an entry for each position, otherwise you get an NPE.
+    // We should fix that.
+    final SortedMap<Integer, ImmutableBitSet> map = Maps.newTreeMap();
+    map.put(0, ImmutableBitSet.of(3));
+    map.put(1, ImmutableBitSet.of());
+    map.put(2, ImmutableBitSet.of(7));
+    map.put(3, ImmutableBitSet.of(4, 12));
+    map.put(4, ImmutableBitSet.of());
+    map.put(5, ImmutableBitSet.of());
+    map.put(6, ImmutableBitSet.of());
+    map.put(7, ImmutableBitSet.of());
+    map.put(8, ImmutableBitSet.of());
+    map.put(9, ImmutableBitSet.of());
+    map.put(10, ImmutableBitSet.of());
+    map.put(11, ImmutableBitSet.of());
+    map.put(12, ImmutableBitSet.of());
+    String original = map.toString();
+    assertThat(ImmutableBitSet.closure(map).toString(),
+        equalTo(
+            "{0={3, 4, 12}, 1={}, 2={7}, 3={3, 4, 12}, 4={4, 12}, 5={}, 6={}, 7={7}, 8={}, 9={}, 10={}, 11={}, 12={4, 12}}"));
+    assertThat("argument modified", map.toString(), equalTo(original));
+  }
+}
+
+// End ImmutableBitSetTest.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoAggregate.java
----------------------------------------------------------------------
diff --git a/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoAggregate.java b/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoAggregate.java
index 9a1744e..81940f7 100644
--- a/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoAggregate.java
+++ b/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoAggregate.java
@@ -26,12 +26,11 @@ import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.fun.SqlSumAggFunction;
 import org.apache.calcite.sql.fun.SqlSumEmptyIsZeroAggFunction;
-import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.Util;
 
 import java.util.AbstractList;
 import java.util.ArrayList;
-import java.util.BitSet;
 import java.util.List;
 
 /**
@@ -46,7 +45,7 @@ public class MongoAggregate
       RelOptCluster cluster,
       RelTraitSet traitSet,
       RelNode child,
-      BitSet groupSet,
+      ImmutableBitSet groupSet,
       List<AggregateCall> aggCalls)
       throws InvalidRelException {
     super(cluster, traitSet, child, groupSet, aggCalls);
@@ -62,7 +61,7 @@ public class MongoAggregate
   }
 
   @Override public Aggregate copy(RelTraitSet traitSet, RelNode input,
-      BitSet groupSet, List<AggregateCall> aggCalls) {
+      ImmutableBitSet groupSet, List<AggregateCall> aggCalls) {
     try {
       return new MongoAggregate(getCluster(), traitSet, input, groupSet,
           aggCalls);
@@ -81,12 +80,12 @@ public class MongoAggregate
     final List<String> outNames = MongoRules.mongoFieldNames(getRowType());
     int i = 0;
     if (groupSet.cardinality() == 1) {
-      final String inName = inNames.get(BitSets.toList(groupSet).get(0));
+      final String inName = inNames.get(groupSet.nth(0));
       list.add("_id: " + MongoRules.maybeQuote("$" + inName));
       ++i;
     } else {
       List<String> keys = new ArrayList<String>();
-      for (int group : BitSets.toIter(groupSet)) {
+      for (int group : groupSet) {
         final String inName = inNames.get(group);
         keys.add(inName + ": " + MongoRules.quote("$" + inName));
         ++i;
@@ -117,7 +116,7 @@ public class MongoAggregate
       fixups = new ArrayList<String>();
       fixups.add("_id: 0");
       i = 0;
-      for (int group : BitSets.toIter(groupSet)) {
+      for (int group : groupSet) {
         fixups.add(
             MongoRules.maybeQuote(outNames.get(group))
             + ": "

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/plus/src/main/java/org/apache/calcite/adapter/tpcds/TpcdsSchema.java
----------------------------------------------------------------------
diff --git a/plus/src/main/java/org/apache/calcite/adapter/tpcds/TpcdsSchema.java b/plus/src/main/java/org/apache/calcite/adapter/tpcds/TpcdsSchema.java
index 0eab5ca..5f240e3 100644
--- a/plus/src/main/java/org/apache/calcite/adapter/tpcds/TpcdsSchema.java
+++ b/plus/src/main/java/org/apache/calcite/adapter/tpcds/TpcdsSchema.java
@@ -30,6 +30,7 @@ import org.apache.calcite.schema.Table;
 import org.apache.calcite.schema.impl.AbstractSchema;
 import org.apache.calcite.schema.impl.AbstractTableQueryable;
 import org.apache.calcite.util.Bug;
+import org.apache.calcite.util.ImmutableBitSet;
 
 import com.google.common.collect.ImmutableMap;
 
@@ -38,7 +39,6 @@ import net.hydromatic.tpcds.TpcdsEntity;
 import net.hydromatic.tpcds.TpcdsTable;
 
 import java.sql.Date;
-import java.util.BitSet;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
@@ -112,7 +112,7 @@ public class TpcdsSchema extends AbstractSchema {
       Bug.upgrade("add row count estimate to TpcdsTable, and use it");
       Integer rowCount = TABLE_ROW_COUNTS.get(tpcdsTable.name);
       assert rowCount != null : tpcdsTable.name;
-      return Statistics.of(rowCount, Collections.<BitSet>emptyList());
+      return Statistics.of(rowCount, Collections.<ImmutableBitSet>emptyList());
     }
 
     public <T> Queryable<T> asQueryable(final QueryProvider queryProvider,


[45/58] [abbrv] [partial] incubator-calcite git commit: [CALCITE-306] Standardize code style for "import package.*; "

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java
index 98b78c3..9b129e5 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java
@@ -14,42 +14,158 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.rules.java;
-
-import net.hydromatic.linq4j.Ord;
-import net.hydromatic.linq4j.expressions.*;
-
-import net.hydromatic.optiq.*;
-import net.hydromatic.optiq.Function;
-import net.hydromatic.optiq.impl.AggregateFunctionImpl;
-import net.hydromatic.optiq.runtime.SqlFunctions;
-
-import org.eigenbase.rel.Aggregation;
-import org.eigenbase.reltype.*;
-import org.eigenbase.rex.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.fun.SqlStdOperatorTable;
-import org.eigenbase.sql.fun.SqlTrimFunction;
-import org.eigenbase.sql.type.SqlTypeName;
-import org.eigenbase.sql.type.SqlTypeUtil;
-import org.eigenbase.sql.validate.SqlUserDefinedAggFunction;
-import org.eigenbase.sql.validate.SqlUserDefinedFunction;
-import org.eigenbase.util.Util;
-import org.eigenbase.util14.DateTimeUtil;
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.linq4j.tree.BlockBuilder;
+import org.apache.calcite.linq4j.tree.BlockStatement;
+import org.apache.calcite.linq4j.tree.ConstantExpression;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.ExpressionType;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.linq4j.tree.MemberExpression;
+import org.apache.calcite.linq4j.tree.OptimizeVisitor;
+import org.apache.calcite.linq4j.tree.ParameterExpression;
+import org.apache.calcite.linq4j.tree.Primitive;
+import org.apache.calcite.linq4j.tree.Types;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeFactoryImpl;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.runtime.SqlFunctions;
+import org.apache.calcite.schema.Function;
+import org.apache.calcite.schema.ImplementableAggFunction;
+import org.apache.calcite.schema.ImplementableFunction;
+import org.apache.calcite.schema.impl.AggregateFunctionImpl;
+import org.apache.calcite.sql.SqlAggFunction;
+import org.apache.calcite.sql.SqlBinaryOperator;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.fun.SqlTrimFunction;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.type.SqlTypeUtil;
+import org.apache.calcite.sql.validate.SqlUserDefinedAggFunction;
+import org.apache.calcite.sql.validate.SqlUserDefinedFunction;
+import org.apache.calcite.util.BuiltInMethod;
+import org.apache.calcite.util.DateTimeUtil;
+import org.apache.calcite.util.Util;
 
 import com.google.common.base.Supplier;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
 
-import java.lang.reflect.*;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.lang.reflect.Type;
 import java.math.BigDecimal;
-import java.util.*;
-
-import static net.hydromatic.linq4j.expressions.ExpressionType.*;
-
-import static net.hydromatic.optiq.DataContext.ROOT;
-
-import static org.eigenbase.sql.fun.SqlStdOperatorTable.*;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.calcite.DataContext.ROOT;
+import static org.apache.calcite.linq4j.tree.ExpressionType.Add;
+import static org.apache.calcite.linq4j.tree.ExpressionType.AndAlso;
+import static org.apache.calcite.linq4j.tree.ExpressionType.Divide;
+import static org.apache.calcite.linq4j.tree.ExpressionType.Equal;
+import static org.apache.calcite.linq4j.tree.ExpressionType.GreaterThan;
+import static org.apache.calcite.linq4j.tree.ExpressionType.GreaterThanOrEqual;
+import static org.apache.calcite.linq4j.tree.ExpressionType.LessThan;
+import static org.apache.calcite.linq4j.tree.ExpressionType.LessThanOrEqual;
+import static org.apache.calcite.linq4j.tree.ExpressionType.Multiply;
+import static org.apache.calcite.linq4j.tree.ExpressionType.Negate;
+import static org.apache.calcite.linq4j.tree.ExpressionType.Not;
+import static org.apache.calcite.linq4j.tree.ExpressionType.NotEqual;
+import static org.apache.calcite.linq4j.tree.ExpressionType.OrElse;
+import static org.apache.calcite.linq4j.tree.ExpressionType.Subtract;
+import static org.apache.calcite.linq4j.tree.ExpressionType.UnaryPlus;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.ABS;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.AND;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.ARRAY_VALUE_CONSTRUCTOR;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.CARDINALITY;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.CASE;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.CAST;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.CEIL;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.CHARACTER_LENGTH;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.CHAR_LENGTH;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.CONCAT;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.COUNT;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.CURRENT_DATE;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.CURRENT_PATH;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.CURRENT_ROLE;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.CURRENT_TIME;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.CURRENT_TIMESTAMP;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.CURRENT_USER;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.DATETIME_PLUS;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.DENSE_RANK;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.DIVIDE;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.DIVIDE_INTEGER;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.ELEMENT;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.EQUALS;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.EXP;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.EXTRACT_DATE;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.FIRST_VALUE;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.FLOOR;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.GREATER_THAN;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.GREATER_THAN_OR_EQUAL;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.INITCAP;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.IS_FALSE;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.IS_NOT_FALSE;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.IS_NOT_NULL;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.IS_NOT_TRUE;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.IS_NULL;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.IS_TRUE;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.ITEM;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.LAG;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.LAST_VALUE;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.LEAD;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.LESS_THAN;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.LESS_THAN_OR_EQUAL;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.LIKE;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.LN;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.LOCALTIME;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.LOCALTIMESTAMP;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.LOG10;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.LOWER;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.MAP_VALUE_CONSTRUCTOR;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.MAX;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.MIN;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.MINUS;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.MOD;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.MULTIPLY;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.NOT;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.NOT_EQUALS;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.NOT_LIKE;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.NOT_SIMILAR_TO;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.NTILE;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.OR;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.OVERLAY;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.PLUS;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.POSITION;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.POWER;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.RANK;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.REINTERPRET;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.ROW_NUMBER;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.SESSION_USER;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.SIMILAR_TO;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.SINGLE_VALUE;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.SLICE;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.SUBSTRING;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.SUM;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.SUM0;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.SYSTEM_USER;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.TRIM;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.UNARY_MINUS;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.UNARY_PLUS;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.UPPER;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.USER;
 
 /**
  * Contains implementations of Rex operators as Java code.
@@ -68,25 +184,24 @@ public class RexImpTable {
 
   private final Map<SqlOperator, CallImplementor> map =
       new HashMap<SqlOperator, CallImplementor>();
-  private final Map<Aggregation, Supplier<? extends AggImplementor>> aggMap =
-      new HashMap<Aggregation, Supplier<? extends AggImplementor>>();
-  private final Map<Aggregation, Supplier<? extends WinAggImplementor>>
-  winAggMap =
-      new HashMap<Aggregation, Supplier<? extends WinAggImplementor>>();
+  private final Map<SqlAggFunction, Supplier<? extends AggImplementor>> aggMap =
+      Maps.newHashMap();
+  private final Map<SqlAggFunction, Supplier<? extends WinAggImplementor>>
+  winAggMap = Maps.newHashMap();
 
   RexImpTable() {
-    defineMethod(UPPER, BuiltinMethod.UPPER.method, NullPolicy.STRICT);
-    defineMethod(LOWER, BuiltinMethod.LOWER.method, NullPolicy.STRICT);
-    defineMethod(INITCAP,  BuiltinMethod.INITCAP.method, NullPolicy.STRICT);
-    defineMethod(SUBSTRING, BuiltinMethod.SUBSTRING.method, NullPolicy.STRICT);
-    defineMethod(CHARACTER_LENGTH, BuiltinMethod.CHAR_LENGTH.method,
+    defineMethod(UPPER, BuiltInMethod.UPPER.method, NullPolicy.STRICT);
+    defineMethod(LOWER, BuiltInMethod.LOWER.method, NullPolicy.STRICT);
+    defineMethod(INITCAP,  BuiltInMethod.INITCAP.method, NullPolicy.STRICT);
+    defineMethod(SUBSTRING, BuiltInMethod.SUBSTRING.method, NullPolicy.STRICT);
+    defineMethod(CHARACTER_LENGTH, BuiltInMethod.CHAR_LENGTH.method,
         NullPolicy.STRICT);
-    defineMethod(CHAR_LENGTH, BuiltinMethod.CHAR_LENGTH.method,
+    defineMethod(CHAR_LENGTH, BuiltInMethod.CHAR_LENGTH.method,
         NullPolicy.STRICT);
-    defineMethod(CONCAT, BuiltinMethod.STRING_CONCAT.method,
+    defineMethod(CONCAT, BuiltInMethod.STRING_CONCAT.method,
         NullPolicy.STRICT);
-    defineMethod(OVERLAY, BuiltinMethod.OVERLAY.method, NullPolicy.STRICT);
-    defineMethod(POSITION, BuiltinMethod.POSITION.method, NullPolicy.STRICT);
+    defineMethod(OVERLAY, BuiltInMethod.OVERLAY.method, NullPolicy.STRICT);
+    defineMethod(POSITION, BuiltInMethod.POSITION.method, NullPolicy.STRICT);
 
     final TrimImplementor trimImplementor = new TrimImplementor();
     defineImplementor(TRIM, NullPolicy.STRICT, trimImplementor, false);
@@ -126,7 +241,7 @@ public class RexImpTable {
     // datetime
     defineImplementor(DATETIME_PLUS, NullPolicy.STRICT,
         new DatetimeArithmeticImplementor(), false);
-    defineMethod(EXTRACT_DATE, BuiltinMethod.UNIX_DATE_EXTRACT.method,
+    defineMethod(EXTRACT_DATE, BuiltInMethod.UNIX_DATE_EXTRACT.method,
         NullPolicy.STRICT);
 
     map.put(IS_NULL, new IsXxxImplementor(null, false));
@@ -138,21 +253,21 @@ public class RexImpTable {
 
     // LIKE and SIMILAR
     final MethodImplementor likeImplementor =
-        new MethodImplementor(BuiltinMethod.LIKE.method);
+        new MethodImplementor(BuiltInMethod.LIKE.method);
     defineImplementor(LIKE, NullPolicy.STRICT, likeImplementor, false);
     defineImplementor(NOT_LIKE, NullPolicy.STRICT,
         NotImplementor.of(likeImplementor), false);
     final MethodImplementor similarImplementor =
-        new MethodImplementor(BuiltinMethod.SIMILAR.method);
+        new MethodImplementor(BuiltInMethod.SIMILAR.method);
     defineImplementor(SIMILAR_TO, NullPolicy.STRICT, similarImplementor, false);
     defineImplementor(NOT_SIMILAR_TO, NullPolicy.STRICT,
         NotImplementor.of(similarImplementor), false);
 
     // Multisets & arrays
-    defineMethod(CARDINALITY, BuiltinMethod.COLLECTION_SIZE.method,
+    defineMethod(CARDINALITY, BuiltInMethod.COLLECTION_SIZE.method,
         NullPolicy.STRICT);
-    defineMethod(SLICE, BuiltinMethod.SLICE.method, NullPolicy.NONE);
-    defineMethod(ELEMENT, BuiltinMethod.ELEMENT.method, NullPolicy.STRICT);
+    defineMethod(SLICE, BuiltInMethod.SLICE.method, NullPolicy.NONE);
+    defineMethod(ELEMENT, BuiltInMethod.ELEMENT.method, NullPolicy.STRICT);
 
     map.put(CASE, new CaseImplementor());
 
@@ -351,7 +466,7 @@ public class RexImpTable {
             NullAs nullAs) {
           switch (nullAs) {
           case NULL:
-            return Expressions.call(BuiltinMethod.NOT.method,
+            return Expressions.call(BuiltInMethod.NOT.method,
                 translator.translateList(call.getOperands(), nullAs));
           default:
             return Expressions.not(
@@ -428,24 +543,22 @@ public class RexImpTable {
       Function udf =
         ((SqlUserDefinedFunction) operator).getFunction();
       if (!(udf instanceof ImplementableFunction)) {
-        throw new IllegalStateException(
-            "User defined function " + operator + " must implement "
-            + "ImplementableFunction");
+        throw new IllegalStateException("User defined function " + operator
+            + " must implement ImplementableFunction");
       }
       return ((ImplementableFunction) udf).getImplementor();
     }
     return map.get(operator);
   }
 
-  public AggImplementor get(final Aggregation aggregation,
+  public AggImplementor get(final SqlAggFunction aggregation,
       boolean forWindowAggregate) {
     if (aggregation instanceof SqlUserDefinedAggFunction) {
       final SqlUserDefinedAggFunction udaf =
           (SqlUserDefinedAggFunction) aggregation;
       if (!(udaf.function instanceof ImplementableAggFunction)) {
-        throw new IllegalStateException(
-            "User defined aggregation " + aggregation + " must implement "
-                + "ImplementableAggFunction");
+        throw new IllegalStateException("User defined aggregation "
+            + aggregation + " must implement ImplementableAggFunction");
       }
       return ((ImplementableAggFunction) udaf.function)
           .getImplementor(forWindowAggregate);
@@ -755,11 +868,11 @@ public class RexImpTable {
         return x;
       case FALSE:
         return Expressions.call(
-            BuiltinMethod.IS_TRUE.method,
+            BuiltInMethod.IS_TRUE.method,
             x);
       case TRUE:
         return Expressions.call(
-            BuiltinMethod.IS_NOT_FALSE.method,
+            BuiltInMethod.IS_NOT_FALSE.method,
             x);
       case IS_NULL:
         return Expressions.equal(x, NULL_EXPR);
@@ -779,19 +892,20 @@ public class RexImpTable {
     return Expressions.constant(null, type);
   }
 
+  /** Implementor for the {@code COUNT} aggregate function. */
   static class CountImplementor extends StrictAggImplementor {
-    @Override
-    public void implementNotNullAdd(AggContext info, AggAddContext add) {
+    @Override public void implementNotNullAdd(AggContext info,
+        AggAddContext add) {
       add.currentBlock().add(Expressions.statement(
           Expressions.postIncrementAssign(add.accumulator().get(0))));
     }
   }
 
+  /** Implementor for the {@code COUNT} windowed aggregate function. */
   static class CountWinImplementor extends StrictWinAggImplementor {
     boolean justFrameRowCount;
 
-    @Override
-    public List<Type> getNotNullState(WinAggContext info) {
+    @Override public List<Type> getNotNullState(WinAggContext info) {
       boolean hasNullable = false;
       for (RelDataType type : info.parameterRelTypes()) {
         if (type.isNullable()) {
@@ -806,8 +920,8 @@ public class RexImpTable {
       return super.getNotNullState(info);
     }
 
-    @Override
-    public void implementNotNullAdd(WinAggContext info, WinAggAddContext add) {
+    @Override public void implementNotNullAdd(WinAggContext info,
+        WinAggAddContext add) {
       if (justFrameRowCount) {
         return;
       }
@@ -815,8 +929,7 @@ public class RexImpTable {
           Expressions.postIncrementAssign(add.accumulator().get(0))));
     }
 
-    @Override
-    protected Expression implementNotNullResult(WinAggContext info,
+    @Override protected Expression implementNotNullResult(WinAggContext info,
         WinAggResultContext result) {
       if (justFrameRowCount) {
         return result.getFrameRowCount();
@@ -825,9 +938,9 @@ public class RexImpTable {
     }
   }
 
+  /** Implementor for the {@code SUM} windowed aggregate function. */
   static class SumImplementor extends StrictAggImplementor {
-    @Override
-    protected void implementNotNullReset(AggContext info,
+    @Override protected void implementNotNullReset(AggContext info,
         AggResetContext reset) {
       Expression start = info.returnType() == BigDecimal.class
           ? Expressions.constant(BigDecimal.ZERO)
@@ -837,8 +950,8 @@ public class RexImpTable {
           reset.accumulator().get(0), start)));
     }
 
-    @Override
-    public void implementNotNullAdd(AggContext info, AggAddContext add) {
+    @Override public void implementNotNullAdd(AggContext info,
+        AggAddContext add) {
       Expression acc = add.accumulator().get(0);
       Expression next;
       if (info.returnType() == BigDecimal.class) {
@@ -850,16 +963,15 @@ public class RexImpTable {
       accAdvance(add, acc, next);
     }
 
-    @Override
-    public Expression implementNotNullResult(AggContext info,
+    @Override public Expression implementNotNullResult(AggContext info,
         AggResultContext result) {
       return super.implementNotNullResult(info, result);
     }
   }
 
+  /** Implementor for the {@code MIN} and {@code MAX} aggregate functions. */
   static class MinMaxImplementor extends StrictAggImplementor {
-    @Override
-    protected void implementNotNullReset(AggContext info,
+    @Override protected void implementNotNullReset(AggContext info,
         AggResetContext reset) {
       Expression acc = reset.accumulator().get(0);
       Primitive p = Primitive.of(acc.getType());
@@ -869,11 +981,11 @@ public class RexImpTable {
           acc, Expressions.constant(inf, acc.getType()))));
     }
 
-    @Override
-    public void implementNotNullAdd(AggContext info, AggAddContext add) {
+    @Override public void implementNotNullAdd(AggContext info,
+        AggAddContext add) {
       Expression acc = add.accumulator().get(0);
       Expression arg = add.arguments().get(0);
-      Aggregation aggregation = info.aggregation();
+      SqlAggFunction aggregation = info.aggregation();
       Expression next = Expressions.call(
           SqlFunctions.class,
           aggregation == MIN ? "lesser" : "greater",
@@ -883,6 +995,7 @@ public class RexImpTable {
     }
   }
 
+  /** Implementor for the {@code SINGLE_VALUE} aggregate function. */
   static class SingleValueImplementor implements AggImplementor {
     public List<Type> getStateType(AggContext info) {
       return Arrays.asList(boolean.class, info.returnType());
@@ -916,6 +1029,7 @@ public class RexImpTable {
     }
   }
 
+  /** Implementor for user-defined aggregate functions. */
   public static class UserDefinedAggReflectiveImplementor
       extends StrictAggImplementor {
     private final AggregateFunctionImpl afi;
@@ -924,16 +1038,14 @@ public class RexImpTable {
       this.afi = afi;
     }
 
-    @Override
-    public List<Type> getNotNullState(AggContext info) {
+    @Override public List<Type> getNotNullState(AggContext info) {
       if (afi.isStatic) {
         return Collections.<Type>singletonList(afi.accumulatorType);
       }
       return Arrays.<Type>asList(afi.accumulatorType, afi.declaringClass);
     }
 
-    @Override
-    protected void implementNotNullReset(AggContext info,
+    @Override protected void implementNotNullReset(AggContext info,
         AggResetContext reset) {
       List<Expression> acc = reset.accumulator();
       if (!afi.isStatic) {
@@ -946,8 +1058,8 @@ public class RexImpTable {
               afi.isStatic ? null : acc.get(1), afi.initMethod))));
     }
 
-    @Override
-    protected void implementNotNullAdd(AggContext info, AggAddContext add) {
+    @Override protected void implementNotNullAdd(AggContext info,
+        AggAddContext add) {
       List<Expression> acc = add.accumulator();
       List<Expression> aggArgs = add.arguments();
       List<Expression> args = new ArrayList<Expression>(aggArgs.size() + 1);
@@ -959,8 +1071,7 @@ public class RexImpTable {
               args))));
     }
 
-    @Override
-    protected Expression implementNotNullResult(AggContext info,
+    @Override protected Expression implementNotNullResult(AggContext info,
         AggResultContext result) {
       List<Expression> acc = result.accumulator();
       return Expressions.call(
@@ -968,9 +1079,9 @@ public class RexImpTable {
     }
   }
 
+  /** Implementor for the {@code RANK} windowed aggregate function. */
   static class RankImplementor extends StrictWinAggImplementor {
-    @Override
-    protected void implementNotNullAdd(WinAggContext info,
+    @Override protected void implementNotNullAdd(WinAggContext info,
         WinAggAddContext add) {
       Expression acc = add.accumulator().get(0);
       // This is an example of the generated code
@@ -1013,8 +1124,7 @@ public class RexImpTable {
       return pos;
     }
 
-    @Override
-    protected Expression implementNotNullResult(
+    @Override protected Expression implementNotNullResult(
         WinAggContext info, WinAggResultContext result) {
       // Rank is 1-based
       return Expressions.add(super.implementNotNullResult(info, result),
@@ -1022,13 +1132,16 @@ public class RexImpTable {
     }
   }
 
+  /** Implementor for the {@code DENSE_RANK} windowed aggregate function. */
   static class DenseRankImplementor extends RankImplementor {
-    @Override
-    protected Expression computeNewRank(Expression acc, WinAggAddContext add) {
+    @Override protected Expression computeNewRank(Expression acc,
+        WinAggAddContext add) {
       return Expressions.add(acc, Expressions.constant(1));
     }
   }
 
+  /** Implementor for the {@code FIRST_VALUE} and {@code LAST_VALUE}
+   * windowed aggregate functions. */
   static class FirstLastValueImplementor implements WinAggImplementor {
     private final SeekType seekType;
 
@@ -1064,18 +1177,22 @@ public class RexImpTable {
     }
   }
 
+  /** Implementor for the {@code FIRST_VALUE} windowed aggregate function. */
   static class FirstValueImplementor extends FirstLastValueImplementor {
     protected FirstValueImplementor() {
       super(SeekType.START);
     }
   }
 
+  /** Implementor for the {@code LAST_VALUE} windowed aggregate function. */
   static class LastValueImplementor extends FirstLastValueImplementor {
     protected LastValueImplementor() {
       super(SeekType.END);
     }
   }
 
+  /** Implementor for the {@code LEAD} and {@code LAG} windowed
+   * aggregate functions. */
   static class LeadLagImplementor implements WinAggImplementor {
     private final boolean isLead;
 
@@ -1144,18 +1261,21 @@ public class RexImpTable {
     }
   }
 
+  /** Implementor for the {@code LEAD} windowed aggregate function. */
   public static class LeadImplementor extends LeadLagImplementor {
     protected LeadImplementor() {
       super(true);
     }
   }
 
+  /** Implementor for the {@code LAG} windowed aggregate function. */
   public static class LagImplementor extends LeadLagImplementor {
     protected LagImplementor() {
       super(false);
     }
   }
 
+  /** Implementor for the {@code NTILE} windowed aggregate function. */
   static class NtileImplementor implements WinAggImplementor {
     public List<Type> getStateType(AggContext info) {
       return Collections.emptyList();
@@ -1196,20 +1316,18 @@ public class RexImpTable {
     }
   }
 
+  /** Implementor for the {@code ROW_NUMBER} windowed aggregate function. */
   static class RowNumberImplementor extends StrictWinAggImplementor {
-    @Override
-    public List<Type> getNotNullState(WinAggContext info) {
+    @Override public List<Type> getNotNullState(WinAggContext info) {
       return Collections.emptyList();
     }
 
-    @Override
-    protected void implementNotNullAdd(WinAggContext info,
+    @Override protected void implementNotNullAdd(WinAggContext info,
         WinAggAddContext add) {
       // no op
     }
 
-    @Override
-    protected Expression implementNotNullResult(
+    @Override protected Expression implementNotNullResult(
         WinAggContext info, WinAggResultContext result) {
       // Window cannot be empty since ROWS/RANGE is not possible for ROW_NUMBER
       return Expressions.add(Expressions.subtract(
@@ -1217,6 +1335,7 @@ public class RexImpTable {
     }
   }
 
+  /** Implementor for the {@code TRIM} function. */
   private static class TrimImplementor implements NotNullImplementor {
     public Expression implement(RexToLixTranslator translator, RexCall call,
         List<Expression> translatedOperands) {
@@ -1224,7 +1343,7 @@ public class RexImpTable {
           ((ConstantExpression) translatedOperands.get(0)).value;
       SqlTrimFunction.Flag flag = (SqlTrimFunction.Flag) value;
       return Expressions.call(
-          BuiltinMethod.TRIM.method,
+          BuiltInMethod.TRIM.method,
           Expressions.constant(
               flag == SqlTrimFunction.Flag.BOTH
               || flag == SqlTrimFunction.Flag.LEADING),
@@ -1236,6 +1355,7 @@ public class RexImpTable {
     }
   }
 
+  /** Implementor for a function that generates calls to a given method. */
   private static class MethodImplementor implements NotNullImplementor {
     private final Method method;
 
@@ -1256,6 +1376,10 @@ public class RexImpTable {
     }
   }
 
+  /** Implementor for SQL functions that generates calls to a given method name.
+   *
+   * <p>Use this, as opposed to {@link MethodImplementor}, if the SQL function
+   * is overloaded; then you can use one implementor for several overloads. */
   private static class MethodNameImplementor implements NotNullImplementor {
     private final String methodName;
 
@@ -1274,6 +1398,7 @@ public class RexImpTable {
     }
   }
 
+  /** Implementor for binary operators. */
   private static class BinaryImplementor implements NotNullImplementor {
     /** Types that can be arguments to comparison operators such as
      * {@code <}. */
@@ -1338,6 +1463,7 @@ public class RexImpTable {
     }
   }
 
+  /** Implementor for unary operators. */
   private static class UnaryImplementor implements NotNullImplementor {
     private final ExpressionType expressionType;
 
@@ -1355,6 +1481,7 @@ public class RexImpTable {
     }
   }
 
+  /** Implementor for the SQL {@code CASE} operator. */
   private static class CaseImplementor implements CallImplementor {
     public Expression implement(RexToLixTranslator translator, RexCall call,
         NullAs nullAs) {
@@ -1399,6 +1526,8 @@ public class RexImpTable {
     }
   }
 
+  /** Implementor for the SQL {@code CAST} function that optimizes if, say, the
+   * argument is already of the desired type. */
   private static class CastOptimizedImplementor implements CallImplementor {
     private final CallImplementor accurate;
 
@@ -1427,6 +1556,7 @@ public class RexImpTable {
     }
   }
 
+  /** Implementor for the SQL {@code CAST} operator. */
   private static class CastImplementor implements NotNullImplementor {
     public Expression implement(
         RexToLixTranslator translator,
@@ -1450,6 +1580,7 @@ public class RexImpTable {
     }
   }
 
+  /** Implementor for the {@code REINTERPRET} internal SQL operator. */
   private static class ReinterpretImplementor implements NotNullImplementor {
     public Expression implement(
         RexToLixTranslator translator,
@@ -1460,6 +1591,7 @@ public class RexImpTable {
     }
   }
 
+  /** Implementor for a value-constructor. */
   private static class ValueConstructorImplementor
       implements CallImplementor {
     public Expression implement(
@@ -1471,6 +1603,7 @@ public class RexImpTable {
     }
   }
 
+  /** Implementor for the {@code ITEM} SQL operator. */
   private static class ItemImplementor
       implements CallImplementor {
     public Expression implement(
@@ -1488,15 +1621,19 @@ public class RexImpTable {
     private MethodImplementor getImplementor(SqlTypeName sqlTypeName) {
       switch (sqlTypeName) {
       case ARRAY:
-        return new MethodImplementor(BuiltinMethod.ARRAY_ITEM.method);
+        return new MethodImplementor(BuiltInMethod.ARRAY_ITEM.method);
       case MAP:
-        return new MethodImplementor(BuiltinMethod.MAP_ITEM.method);
+        return new MethodImplementor(BuiltInMethod.MAP_ITEM.method);
       default:
-        return new MethodImplementor(BuiltinMethod.ANY_ITEM.method);
+        return new MethodImplementor(BuiltInMethod.ANY_ITEM.method);
       }
     }
   }
 
+    /** Implementor for SQL system functions.
+     *
+     * <p>Several of these are represented internally as constant values, set
+     * per execution. */
   private static class SystemFunctionImplementor
       implements CallImplementor {
     public Expression implement(
@@ -1522,15 +1659,15 @@ public class RexImpTable {
         // the empty string because a role has to be set explicitly.
         return Expressions.constant("");
       } else if (op == CURRENT_TIMESTAMP) {
-        return Expressions.call(BuiltinMethod.CURRENT_TIMESTAMP.method, ROOT);
+        return Expressions.call(BuiltInMethod.CURRENT_TIMESTAMP.method, ROOT);
       } else if (op == CURRENT_TIME) {
-        return Expressions.call(BuiltinMethod.CURRENT_TIME.method, ROOT);
+        return Expressions.call(BuiltInMethod.CURRENT_TIME.method, ROOT);
       } else if (op == CURRENT_DATE) {
-        return Expressions.call(BuiltinMethod.CURRENT_DATE.method, ROOT);
+        return Expressions.call(BuiltInMethod.CURRENT_DATE.method, ROOT);
       } else if (op == LOCALTIMESTAMP) {
-        return Expressions.call(BuiltinMethod.LOCAL_TIMESTAMP.method, ROOT);
+        return Expressions.call(BuiltInMethod.LOCAL_TIMESTAMP.method, ROOT);
       } else if (op == LOCALTIME) {
-        return Expressions.call(BuiltinMethod.LOCAL_TIME.method, ROOT);
+        return Expressions.call(BuiltInMethod.LOCAL_TIME.method, ROOT);
       } else {
         throw new AssertionError("unknown function " + op);
       }
@@ -1572,6 +1709,7 @@ public class RexImpTable {
     }
   }
 
+  /** Implementor for the {@code NOT} operator. */
   private static class NotImplementor implements NotNullImplementor {
     private final NotNullImplementor implementor;
 
@@ -1593,6 +1731,7 @@ public class RexImpTable {
     }
   }
 
+  /** Implementor for various datetime arithmetic. */
   private static class DatetimeArithmeticImplementor
       implements NotNullImplementor {
     public Expression implement(RexToLixTranslator translator, RexCall call,

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/RexToLixTranslator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/RexToLixTranslator.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/RexToLixTranslator.java
index 82ee068..da29cce 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/RexToLixTranslator.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/RexToLixTranslator.java
@@ -14,32 +14,58 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.rules.java;
-
-import net.hydromatic.avatica.ByteString;
-
-import net.hydromatic.linq4j.expressions.*;
-
-import net.hydromatic.optiq.BuiltinMethod;
-import net.hydromatic.optiq.DataContext;
-import net.hydromatic.optiq.impl.java.JavaTypeFactory;
-import net.hydromatic.optiq.runtime.SqlFunctions;
-
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.reltype.RelDataTypeFactoryImpl;
-import org.eigenbase.rex.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.util.*;
-import org.eigenbase.util14.DateTimeUtil;
-
-import java.lang.reflect.*;
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.DataContext;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.avatica.ByteString;
+import org.apache.calcite.linq4j.tree.BlockBuilder;
+import org.apache.calcite.linq4j.tree.ConstantExpression;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.ExpressionType;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.linq4j.tree.ParameterExpression;
+import org.apache.calcite.linq4j.tree.Primitive;
+import org.apache.calcite.linq4j.tree.UnaryExpression;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactoryImpl;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexDynamicParam;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexLocalRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.SqlIntervalQualifier;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.util.BuiltInMethod;
+import org.apache.calcite.util.ControlFlowException;
+import org.apache.calcite.util.DateTimeUtil;
+import org.apache.calcite.util.NlsString;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
+
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.lang.reflect.Type;
 import java.math.BigDecimal;
-import java.util.*;
-
-import static org.eigenbase.sql.fun.SqlStdOperatorTable.*;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.CHARACTER_LENGTH;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.CHAR_LENGTH;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.SUBSTRING;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.UPPER;
 
 /**
- * Translates {@link org.eigenbase.rex.RexNode REX expressions} to
+ * Translates {@link org.apache.calcite.rex.RexNode REX expressions} to
  * {@link Expression linq4j expressions}.
  */
 public class RexToLixTranslator {
@@ -182,7 +208,7 @@ public class RexToLixTranslator {
       case CHAR:
       case VARCHAR:
         convert =
-            Expressions.call(BuiltinMethod.STRING_TO_DATE.method, operand);
+            Expressions.call(BuiltInMethod.STRING_TO_DATE.method, operand);
       }
       break;
     case TIME:
@@ -190,7 +216,7 @@ public class RexToLixTranslator {
       case CHAR:
       case VARCHAR:
         convert =
-            Expressions.call(BuiltinMethod.STRING_TO_TIME.method, operand);
+            Expressions.call(BuiltInMethod.STRING_TO_TIME.method, operand);
       }
       break;
     case TIMESTAMP:
@@ -198,7 +224,7 @@ public class RexToLixTranslator {
       case CHAR:
       case VARCHAR:
         convert =
-            Expressions.call(BuiltinMethod.STRING_TO_TIMESTAMP.method, operand);
+            Expressions.call(BuiltInMethod.STRING_TO_TIMESTAMP.method, operand);
       }
       break;
     case BOOLEAN:
@@ -206,7 +232,7 @@ public class RexToLixTranslator {
       case CHAR:
       case VARCHAR:
         convert = Expressions.call(
-            BuiltinMethod.STRING_TO_BOOLEAN.method,
+            BuiltInMethod.STRING_TO_BOOLEAN.method,
             operand);
       }
       break;
@@ -219,28 +245,28 @@ public class RexToLixTranslator {
         convert = RexImpTable.optimize2(
             operand,
             Expressions.call(
-                BuiltinMethod.UNIX_DATE_TO_STRING.method,
+                BuiltInMethod.UNIX_DATE_TO_STRING.method,
                 operand));
         break;
       case TIME:
         convert = RexImpTable.optimize2(
             operand,
             Expressions.call(
-                BuiltinMethod.UNIX_TIME_TO_STRING.method,
+                BuiltInMethod.UNIX_TIME_TO_STRING.method,
                 operand));
         break;
       case TIMESTAMP:
         convert = RexImpTable.optimize2(
             operand,
             Expressions.call(
-                BuiltinMethod.UNIX_TIMESTAMP_TO_STRING.method,
+                BuiltInMethod.UNIX_TIMESTAMP_TO_STRING.method,
                 operand));
         break;
       case INTERVAL_YEAR_MONTH:
         convert = RexImpTable.optimize2(
             operand,
             Expressions.call(
-                BuiltinMethod.INTERVAL_YEAR_MONTH_TO_STRING.method,
+                BuiltInMethod.INTERVAL_YEAR_MONTH_TO_STRING.method,
                 operand,
                 Expressions.constant(interval.foo())));
         break;
@@ -248,7 +274,7 @@ public class RexToLixTranslator {
         convert = RexImpTable.optimize2(
             operand,
             Expressions.call(
-                BuiltinMethod.INTERVAL_DAY_TIME_TO_STRING.method,
+                BuiltInMethod.INTERVAL_DAY_TIME_TO_STRING.method,
                 operand,
                 Expressions.constant(interval.foo()),
                 Expressions.constant(
@@ -259,7 +285,7 @@ public class RexToLixTranslator {
         convert = RexImpTable.optimize2(
             operand,
             Expressions.call(
-                BuiltinMethod.BOOLEAN_TO_STRING.method,
+                BuiltInMethod.BOOLEAN_TO_STRING.method,
                 operand));
         break;
       }
@@ -273,14 +299,14 @@ public class RexToLixTranslator {
       switch (targetType.getSqlTypeName()) {
       case VARCHAR:
         convert = Expressions.call(
-            BuiltinMethod.RTRIM.method, convert);
+            BuiltInMethod.RTRIM.method, convert);
       }
       break;
     case BINARY:
       switch (targetType.getSqlTypeName()) {
       case VARBINARY:
         convert = Expressions.call(
-            BuiltinMethod.RTRIM.method, convert);
+            BuiltInMethod.RTRIM.method, convert);
       }
       break;
     }
@@ -309,7 +335,7 @@ public class RexToLixTranslator {
         default:
           convert =
               Expressions.call(
-                  BuiltinMethod.TRUNCATE.method,
+                  BuiltInMethod.TRUNCATE.method,
                   convert,
                   Expressions.constant(targetPrecision));
         }
@@ -323,7 +349,7 @@ public class RexToLixTranslator {
       if (targetScale < sourceType.getScale()) {
         convert =
             Expressions.call(
-                BuiltinMethod.ROUND_LONG.method,
+                BuiltInMethod.ROUND_LONG.method,
                 convert,
                 Expressions.constant(
                     (long) Math.pow(10, 3 - targetScale)));
@@ -410,7 +436,7 @@ public class RexToLixTranslator {
   }
 
   /** Dereferences an expression if it is a
-   * {@link org.eigenbase.rex.RexLocalRef}. */
+   * {@link org.apache.calcite.rex.RexLocalRef}. */
   public RexNode deref(RexNode expr) {
     if (expr instanceof RexLocalRef) {
       RexLocalRef ref = (RexLocalRef) expr;
@@ -443,7 +469,7 @@ public class RexToLixTranslator {
         convert(
             Expressions.call(
                 DataContext.ROOT,
-                BuiltinMethod.DATA_CONTEXT_GET.method,
+                BuiltInMethod.DATA_CONTEXT_GET.method,
                 Expressions.constant("?" + expr.getIndex())),
             storageType));
   }
@@ -451,7 +477,7 @@ public class RexToLixTranslator {
   /** Translates a literal.
    *
    * @throws AlwaysNull if literal is null but {@code nullAs} is
-   * {@link net.hydromatic.optiq.rules.java.RexImpTable.NullAs#NOT_POSSIBLE}.
+   * {@link org.apache.calcite.adapter.enumerable.RexImpTable.NullAs#NOT_POSSIBLE}.
    */
   public static Expression translateLiteral(
       RexLiteral literal,
@@ -551,7 +577,7 @@ public class RexToLixTranslator {
    * This might be suboptimal in terms of additional box-unbox when you use
    * the translation later.
    * If you know the java class that will be used to store the results, use
-   * {@link net.hydromatic.optiq.rules.java.RexToLixTranslator#translateList(java.util.List, java.util.List)}
+   * {@link org.apache.calcite.adapter.enumerable.RexToLixTranslator#translateList(java.util.List, java.util.List)}
    * version.
    *
    * @param operandList list of RexNodes to translate
@@ -798,7 +824,7 @@ public class RexToLixTranslator {
             Expressions.statement(
                 Expressions.call(
                     map,
-                    BuiltinMethod.MAP_PUT.method,
+                    BuiltInMethod.MAP_PUT.method,
                     Expressions.box(translate(key)),
                     Expressions.box(translate(value)))));
       }
@@ -814,7 +840,7 @@ public class RexToLixTranslator {
             Expressions.statement(
                 Expressions.call(
                     lyst,
-                    BuiltinMethod.COLLECTION_ADD.method,
+                    BuiltInMethod.COLLECTION_ADD.method,
                     Expressions.box(translate(value)))));
       }
       return lyst;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/StrictAggImplementor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/StrictAggImplementor.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/StrictAggImplementor.java
index d4198b0..ae2ae4c 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/StrictAggImplementor.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/StrictAggImplementor.java
@@ -14,20 +14,29 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.rules.java;
-
-import net.hydromatic.linq4j.expressions.*;
-
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.rex.RexNode;
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.linq4j.tree.BlockBuilder;
+import org.apache.calcite.linq4j.tree.BlockStatement;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.linq4j.tree.ParameterExpression;
+import org.apache.calcite.linq4j.tree.Primitive;
+import org.apache.calcite.linq4j.tree.Types;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexNode;
 
 import java.lang.reflect.Type;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
 
 /**
  * The base implementation of strict aggregate function.
- * @see net.hydromatic.optiq.rules.java.RexImpTable.CountImplementor
- * @see net.hydromatic.optiq.rules.java.RexImpTable.SumImplementor
+ * @see org.apache.calcite.adapter.enumerable.RexImpTable.CountImplementor
+ * @see org.apache.calcite.adapter.enumerable.RexImpTable.SumImplementor
  */
 public abstract class StrictAggImplementor implements AggImplementor {
   private boolean needTrackEmptySet;
@@ -181,3 +190,5 @@ public abstract class StrictAggImplementor implements AggImplementor {
     return result.accumulator().get(0);
   }
 }
+
+// End StrictAggImplementor.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/StrictWinAggImplementor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/StrictWinAggImplementor.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/StrictWinAggImplementor.java
index ba9d4be..f2b0790 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/StrictWinAggImplementor.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/StrictWinAggImplementor.java
@@ -14,18 +14,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.rules.java;
+package org.apache.calcite.adapter.enumerable;
 
-import net.hydromatic.linq4j.expressions.Expression;
+import org.apache.calcite.linq4j.tree.Expression;
 
 import java.lang.reflect.Type;
 import java.util.List;
 
 /**
  * The base implementation of strict window aggregate function.
- * @see net.hydromatic.optiq.rules.java.RexImpTable.FirstLastValueImplementor
- * @see net.hydromatic.optiq.rules.java.RexImpTable.RankImplementor
- * @see net.hydromatic.optiq.rules.java.RexImpTable.RowNumberImplementor
+ * @see org.apache.calcite.adapter.enumerable.RexImpTable.FirstLastValueImplementor
+ * @see org.apache.calcite.adapter.enumerable.RexImpTable.RankImplementor
+ * @see org.apache.calcite.adapter.enumerable.RexImpTable.RowNumberImplementor
  */
 public abstract class StrictWinAggImplementor extends StrictAggImplementor
     implements WinAggImplementor {
@@ -50,29 +50,25 @@ public abstract class StrictWinAggImplementor extends StrictAggImplementor
     return super.implementNotNullResult(info, result);
   }
 
-  @Override
-  protected final void implementNotNullAdd(AggContext info, AggAddContext add) {
+  @Override protected final void implementNotNullAdd(AggContext info,
+      AggAddContext add) {
     implementNotNullAdd((WinAggContext) info, (WinAggAddContext) add);
   }
 
-  @Override
-  protected boolean nonDefaultOnEmptySet(AggContext info) {
+  @Override protected boolean nonDefaultOnEmptySet(AggContext info) {
     return nonDefaultOnEmptySet((WinAggContext) info);
   }
 
-  @Override
-  public final List<Type> getNotNullState(AggContext info) {
+  @Override public final List<Type> getNotNullState(AggContext info) {
     return getNotNullState((WinAggContext) info);
   }
 
-  @Override
-  protected final void implementNotNullReset(AggContext info,
+  @Override protected final void implementNotNullReset(AggContext info,
       AggResetContext reset) {
     implementNotNullReset((WinAggContext) info, (WinAggResetContext) reset);
   }
 
-  @Override
-  protected final Expression implementNotNullResult(AggContext info,
+  @Override protected final Expression implementNotNullResult(AggContext info,
       AggResultContext result) {
     return implementNotNullResult((WinAggContext) info,
         (WinAggResultContext) result);
@@ -82,3 +78,5 @@ public abstract class StrictWinAggImplementor extends StrictAggImplementor
     return true;
   }
 }
+
+// End StrictWinAggImplementor.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/WinAggAddContext.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/WinAggAddContext.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/WinAggAddContext.java
index ca2dca1..13717e5 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/WinAggAddContext.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/WinAggAddContext.java
@@ -14,13 +14,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.rules.java;
+package org.apache.calcite.adapter.enumerable;
 
-import net.hydromatic.linq4j.expressions.Expression;
+import org.apache.calcite.linq4j.tree.Expression;
 
 /**
- * Information for a call to {@link AggImplementor#implementAdd(AggContext, AggAddContext)}.
- * {@link WinAggAddContext} is used when implementing windowed aggregate.
+ * Information for a call to
+ * {@link AggImplementor#implementAdd(AggContext, AggAddContext)}.
+ *
+ * <p>{@link WinAggAddContext} is used when implementing windowed aggregate.
  * Typically, the aggregation implementation will use {@link #arguments()}
  * or {@link #rexArguments()} to update aggregate value.
  * @see AggAddContext

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/WinAggContext.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/WinAggContext.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/WinAggContext.java
index c0fc790..db0a6cd 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/WinAggContext.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/WinAggContext.java
@@ -14,11 +14,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.rules.java;
+package org.apache.calcite.adapter.enumerable;
 
 /**
  * Marker interface to allow
- * {@link net.hydromatic.optiq.rules.java.AggImplementor}
+ * {@link org.apache.calcite.adapter.enumerable.AggImplementor}
  * to tell if it is used in regular or windowed context.
  */
 public interface WinAggContext extends AggContext {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/WinAggFrameContext.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/WinAggFrameContext.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/WinAggFrameContext.java
index 0848354..73782a3 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/WinAggFrameContext.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/WinAggFrameContext.java
@@ -14,16 +14,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.rules.java;
+package org.apache.calcite.adapter.enumerable;
 
-import net.hydromatic.linq4j.expressions.Expression;
+import org.apache.calcite.linq4j.tree.Expression;
 
 /**
  * Provides information on the current window.
  *
  * <p>All the indexes are ready to be used in
- * {@link WinAggResultContext#arguments(net.hydromatic.linq4j.expressions.Expression)},
- * {@link WinAggFrameResultContext#rowTranslator(net.hydromatic.linq4j.expressions.Expression)}
+ * {@link WinAggResultContext#arguments(org.apache.calcite.linq4j.tree.Expression)},
+ * {@link WinAggFrameResultContext#rowTranslator(org.apache.calcite.linq4j.tree.Expression)}
  * and similar methods.
  */
 public interface WinAggFrameContext {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/WinAggFrameResultContext.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/WinAggFrameResultContext.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/WinAggFrameResultContext.java
index d1e867e..f0b4d87 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/WinAggFrameResultContext.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/WinAggFrameResultContext.java
@@ -14,9 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.rules.java;
+package org.apache.calcite.adapter.enumerable;
 
-import net.hydromatic.linq4j.expressions.Expression;
+import org.apache.calcite.linq4j.tree.Expression;
 
 /**
  * Provides information on the current window when computing the result of

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/WinAggImplementor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/WinAggImplementor.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/WinAggImplementor.java
index af89e1d..7d2489f 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/WinAggImplementor.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/WinAggImplementor.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.rules.java;
+package org.apache.calcite.adapter.enumerable;
 
 /**
  * Implements a windowed aggregate function by generating expressions to
@@ -25,10 +25,10 @@ package net.hydromatic.optiq.rules.java;
  * <p>This interface does not define new methods: window-specific
  * sub-interfaces are passed when implementing window aggregate.
  *
- * @see net.hydromatic.optiq.rules.java.StrictWinAggImplementor
- * @see net.hydromatic.optiq.rules.java.RexImpTable.FirstLastValueImplementor
- * @see net.hydromatic.optiq.rules.java.RexImpTable.RankImplementor
- * @see net.hydromatic.optiq.rules.java.RexImpTable.RowNumberImplementor
+ * @see org.apache.calcite.adapter.enumerable.StrictWinAggImplementor
+ * @see org.apache.calcite.adapter.enumerable.RexImpTable.FirstLastValueImplementor
+ * @see org.apache.calcite.adapter.enumerable.RexImpTable.RankImplementor
+ * @see org.apache.calcite.adapter.enumerable.RexImpTable.RowNumberImplementor
  */
 public interface WinAggImplementor extends AggImplementor {
   /**

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/WinAggResetContext.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/WinAggResetContext.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/WinAggResetContext.java
index 691b7ec..9dee01e 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/WinAggResetContext.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/WinAggResetContext.java
@@ -14,13 +14,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.rules.java;
+package org.apache.calcite.adapter.enumerable;
 
 /**
- * Information for a call to {@link AggImplementor#implementReset(AggContext, AggResetContext)}.
- * {@link AggResetContext} provides access to the accumulator variables
+ * Information for a call to
+ * {@link AggImplementor#implementReset(AggContext, AggResetContext)}.
+ *
+ * <p>The {@link AggResetContext} provides access to the accumulator variables
  * that should be reset.
- * Note: the very first reset of windowed aggregates is performed with null
+ *
+ * <p>Note: the very first reset of windowed aggregates is performed with null
  * knowledge of indices and row count in the partition.
  * In other words, the implementation should treat indices and partition row
  * count as a hint to pre-size the collections.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/WinAggResultContext.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/WinAggResultContext.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/WinAggResultContext.java
index 08db039..3b83b19 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/WinAggResultContext.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/WinAggResultContext.java
@@ -14,27 +14,28 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.rules.java;
+package org.apache.calcite.adapter.enumerable;
 
-import net.hydromatic.linq4j.expressions.Expression;
-
-import org.eigenbase.rex.RexNode;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.rex.RexNode;
 
 import java.util.List;
 
 /**
- * Information for a call to {@link AggImplementor#implementResult(AggContext, AggResultContext)}
- * Typically, the aggregation implementation will convert {@link #accumulator()}
- * to the resulting value of the aggregation.
- * The implementation MUST NOT destroy the contents of {@link #accumulator()}.
+ * Information for a call to
+ * {@link AggImplementor#implementResult(AggContext, AggResultContext)}.
+ *
+ * <p>Typically, the aggregation implementation will convert
+ * {@link #accumulator()} to the resulting value of the aggregation.  The
+ * implementation MUST NOT destroy the contents of {@link #accumulator()}.
  */
 public interface WinAggResultContext extends AggResultContext,
     WinAggFrameResultContext {
   /**
-   * Returns {@link org.eigenbase.rex.RexNode} representation of arguments.
+   * Returns {@link org.apache.calcite.rex.RexNode} representation of arguments.
    * This can be useful for manual translation of required arguments with
    * different {@link NullPolicy}.
-   * @return {@link org.eigenbase.rex.RexNode} representation of arguments
+   * @return {@link org.apache.calcite.rex.RexNode} representation of arguments
    */
   List<RexNode> rexArguments();
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/AggAddContextImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/AggAddContextImpl.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/AggAddContextImpl.java
index 103ee04..18ba8cc 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/AggAddContextImpl.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/AggAddContextImpl.java
@@ -14,17 +14,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.rules.java.impl;
+package org.apache.calcite.adapter.enumerable.impl;
 
-import net.hydromatic.linq4j.expressions.BlockBuilder;
-import net.hydromatic.linq4j.expressions.Expression;
-
-import net.hydromatic.optiq.rules.java.AggAddContext;
+import org.apache.calcite.adapter.enumerable.AggAddContext;
+import org.apache.calcite.linq4j.tree.BlockBuilder;
+import org.apache.calcite.linq4j.tree.Expression;
 
 import java.util.List;
 
 /**
- * Implementation of {@link net.hydromatic.optiq.rules.java.AggAddContext}.
+ * Implementation of
+ * {@link org.apache.calcite.adapter.enumerable.AggAddContext}.
  */
 public abstract class AggAddContextImpl extends AggResultContextImpl
     implements AggAddContext {
@@ -36,3 +36,5 @@ public abstract class AggAddContextImpl extends AggResultContextImpl
     return rowTranslator().translateList(rexArguments());
   }
 }
+
+// End AggAddContextImpl.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/AggResetContextImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/AggResetContextImpl.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/AggResetContextImpl.java
index 32ce00e..3e66705 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/AggResetContextImpl.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/AggResetContextImpl.java
@@ -14,18 +14,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.rules.java.impl;
+package org.apache.calcite.adapter.enumerable.impl;
 
-import net.hydromatic.linq4j.expressions.BlockBuilder;
-import net.hydromatic.linq4j.expressions.Expression;
-
-import net.hydromatic.optiq.rules.java.AggResetContext;
-import net.hydromatic.optiq.rules.java.NestedBlockBuilderImpl;
+import org.apache.calcite.adapter.enumerable.AggResetContext;
+import org.apache.calcite.adapter.enumerable.NestedBlockBuilderImpl;
+import org.apache.calcite.linq4j.tree.BlockBuilder;
+import org.apache.calcite.linq4j.tree.Expression;
 
 import java.util.List;
 
 /**
- * Implementation of {@link net.hydromatic.optiq.rules.java.AggResetContext}
+ * Implementation of
+ * {@link org.apache.calcite.adapter.enumerable.AggResetContext}
  */
 public class AggResetContextImpl extends NestedBlockBuilderImpl
     implements AggResetContext {
@@ -47,4 +47,4 @@ public class AggResetContextImpl extends NestedBlockBuilderImpl
   }
 }
 
-// End AggResetContext.java
+// End AggResetContextImpl.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/AggResultContextImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/AggResultContextImpl.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/AggResultContextImpl.java
index 8a93245..1da28c6 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/AggResultContextImpl.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/AggResultContextImpl.java
@@ -14,17 +14,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.rules.java.impl;
+package org.apache.calcite.adapter.enumerable.impl;
 
-import net.hydromatic.linq4j.expressions.BlockBuilder;
-import net.hydromatic.linq4j.expressions.Expression;
-
-import net.hydromatic.optiq.rules.java.AggResultContext;
+import org.apache.calcite.adapter.enumerable.AggResultContext;
+import org.apache.calcite.linq4j.tree.BlockBuilder;
+import org.apache.calcite.linq4j.tree.Expression;
 
 import java.util.List;
 
 /**
- * Implementation of {@link net.hydromatic.optiq.rules.java.AggResultContext}
+ * Implementation of
+ * {@link org.apache.calcite.adapter.enumerable.AggResultContext}
  */
 public class AggResultContextImpl extends AggResetContextImpl
     implements AggResultContext {
@@ -40,4 +40,4 @@ public class AggResultContextImpl extends AggResetContextImpl
   }
 }
 
-// End AggResultContext.java
+// End AggResultContextImpl.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/WinAggAddContextImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/WinAggAddContextImpl.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/WinAggAddContextImpl.java
index 167a5be..adcada8 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/WinAggAddContextImpl.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/WinAggAddContextImpl.java
@@ -14,23 +14,23 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.rules.java.impl;
+package org.apache.calcite.adapter.enumerable.impl;
 
-import net.hydromatic.linq4j.expressions.BlockBuilder;
-import net.hydromatic.linq4j.expressions.Expression;
-import net.hydromatic.linq4j.expressions.Expressions;
-
-import net.hydromatic.optiq.rules.java.RexToLixTranslator;
-import net.hydromatic.optiq.rules.java.WinAggAddContext;
-import net.hydromatic.optiq.rules.java.WinAggFrameResultContext;
-import net.hydromatic.optiq.rules.java.WinAggImplementor;
+import org.apache.calcite.adapter.enumerable.RexToLixTranslator;
+import org.apache.calcite.adapter.enumerable.WinAggAddContext;
+import org.apache.calcite.adapter.enumerable.WinAggFrameResultContext;
+import org.apache.calcite.adapter.enumerable.WinAggImplementor;
+import org.apache.calcite.linq4j.tree.BlockBuilder;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.Expressions;
 
 import com.google.common.base.Function;
 
 import java.util.List;
 
 /**
- * Implementation of {@link net.hydromatic.optiq.rules.java.WinAggAddContext}.
+ * Implementation of
+ * {@link org.apache.calcite.adapter.enumerable.WinAggAddContext}.
  */
 public abstract class WinAggAddContextImpl extends WinAggResultContextImpl
     implements WinAggAddContext {
@@ -49,4 +49,4 @@ public abstract class WinAggAddContextImpl extends WinAggResultContextImpl
   }
 }
 
-// End WinAggAddContext.java
+// End WinAggAddContextImpl.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/WinAggResetContextImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/WinAggResetContextImpl.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/WinAggResetContextImpl.java
index f130fd6..0145f80 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/WinAggResetContextImpl.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/WinAggResetContextImpl.java
@@ -14,17 +14,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.rules.java.impl;
+package org.apache.calcite.adapter.enumerable.impl;
 
-import net.hydromatic.linq4j.expressions.BlockBuilder;
-import net.hydromatic.linq4j.expressions.Expression;
-
-import net.hydromatic.optiq.rules.java.WinAggResetContext;
+import org.apache.calcite.adapter.enumerable.WinAggResetContext;
+import org.apache.calcite.linq4j.tree.BlockBuilder;
+import org.apache.calcite.linq4j.tree.Expression;
 
 import java.util.List;
 
 /**
- * Implementation of {@link net.hydromatic.optiq.rules.java.WinAggResetContext}.
+ * Implementation of
+ * {@link org.apache.calcite.adapter.enumerable.WinAggResetContext}.
  */
 public class WinAggResetContextImpl extends AggResetContextImpl
     implements WinAggResetContext {
@@ -86,3 +86,4 @@ public class WinAggResetContextImpl extends AggResetContextImpl
   }
 }
 
+// End WinAggResetContextImpl.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/WinAggResultContextImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/WinAggResultContextImpl.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/WinAggResultContextImpl.java
index 2baabfc..d4a5c6f 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/WinAggResultContextImpl.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/WinAggResultContextImpl.java
@@ -14,22 +14,22 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.rules.java.impl;
+package org.apache.calcite.adapter.enumerable.impl;
 
-import net.hydromatic.linq4j.expressions.BlockBuilder;
-import net.hydromatic.linq4j.expressions.Expression;
-
-import net.hydromatic.optiq.rules.java.RexToLixTranslator;
-import net.hydromatic.optiq.rules.java.WinAggFrameResultContext;
-import net.hydromatic.optiq.rules.java.WinAggImplementor;
-import net.hydromatic.optiq.rules.java.WinAggResultContext;
+import org.apache.calcite.adapter.enumerable.RexToLixTranslator;
+import org.apache.calcite.adapter.enumerable.WinAggFrameResultContext;
+import org.apache.calcite.adapter.enumerable.WinAggImplementor;
+import org.apache.calcite.adapter.enumerable.WinAggResultContext;
+import org.apache.calcite.linq4j.tree.BlockBuilder;
+import org.apache.calcite.linq4j.tree.Expression;
 
 import com.google.common.base.Function;
 
 import java.util.List;
 
 /**
- * Implementation of {@link net.hydromatic.optiq.rules.java.WinAggResultContext}.
+ * Implementation of
+ * {@link org.apache.calcite.adapter.enumerable.WinAggResultContext}.
  */
 public abstract class WinAggResultContextImpl extends AggResultContextImpl
     implements WinAggResultContext {
@@ -104,4 +104,4 @@ public abstract class WinAggResultContextImpl extends AggResultContextImpl
   }
 }
 
-// End WinAggResultContext.java
+// End WinAggResultContextImpl.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/package-info.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/package-info.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/package-info.java
index 740f248..d1df993 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/package-info.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/package-info.java
@@ -16,8 +16,8 @@
  */
 
 /**
- * Optiq-specific classes for implementation of regular and window aggregates.
+ * Calcite-specific classes for implementation of regular and window aggregates.
  */
-package net.hydromatic.optiq.rules.java.impl;
+package org.apache.calcite.adapter.enumerable.impl;
 
 // End package-info.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/package-info.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/package-info.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/package-info.java
index 695a470..5751e7c 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/package-info.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/package-info.java
@@ -18,6 +18,6 @@
 /**
  * Query optimizer rules for Java calling convention.
  */
-package net.hydromatic.optiq.rules.java;
+package org.apache.calcite.adapter.enumerable;
 
 // End package-info.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/java/AbstractQueryableTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/java/AbstractQueryableTable.java b/core/src/main/java/org/apache/calcite/adapter/java/AbstractQueryableTable.java
index 412d7d0..5cb7b82 100644
--- a/core/src/main/java/org/apache/calcite/adapter/java/AbstractQueryableTable.java
+++ b/core/src/main/java/org/apache/calcite/adapter/java/AbstractQueryableTable.java
@@ -14,17 +14,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl.java;
+package org.apache.calcite.adapter.java;
 
-import net.hydromatic.linq4j.expressions.Expression;
-
-import net.hydromatic.optiq.*;
-import net.hydromatic.optiq.impl.AbstractTable;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.schema.QueryableTable;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.schema.Schemas;
+import org.apache.calcite.schema.impl.AbstractTable;
 
 import java.lang.reflect.Type;
 
 /**
- * Abstract base class for implementing {@link net.hydromatic.optiq.Table}.
+ * Abstract base class for implementing {@link org.apache.calcite.schema.Table}.
  */
 public abstract class AbstractQueryableTable extends AbstractTable
     implements QueryableTable {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/java/JavaTypeFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/java/JavaTypeFactory.java b/core/src/main/java/org/apache/calcite/adapter/java/JavaTypeFactory.java
index 3993fdd..205b391 100644
--- a/core/src/main/java/org/apache/calcite/adapter/java/JavaTypeFactory.java
+++ b/core/src/main/java/org/apache/calcite/adapter/java/JavaTypeFactory.java
@@ -14,10 +14,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl.java;
+package org.apache.calcite.adapter.java;
 
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.reltype.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
 
 import java.lang.reflect.Type;
 import java.util.List;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/java/ReflectiveSchema.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/java/ReflectiveSchema.java b/core/src/main/java/org/apache/calcite/adapter/java/ReflectiveSchema.java
index 9dec68a..27b5151 100644
--- a/core/src/main/java/org/apache/calcite/adapter/java/ReflectiveSchema.java
+++ b/core/src/main/java/org/apache/calcite/adapter/java/ReflectiveSchema.java
@@ -14,26 +14,48 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl.java;
+package org.apache.calcite.adapter.java;
 
-import net.hydromatic.linq4j.*;
-import net.hydromatic.linq4j.expressions.*;
+import org.apache.calcite.linq4j.Enumerable;
+import org.apache.calcite.linq4j.Enumerator;
+import org.apache.calcite.linq4j.Linq4j;
+import org.apache.calcite.linq4j.QueryProvider;
+import org.apache.calcite.linq4j.Queryable;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.linq4j.tree.Primitive;
+import org.apache.calcite.linq4j.tree.Types;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.schema.Function;
+import org.apache.calcite.schema.Schema;
+import org.apache.calcite.schema.SchemaFactory;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.schema.Schemas;
+import org.apache.calcite.schema.Statistic;
+import org.apache.calcite.schema.Statistics;
+import org.apache.calcite.schema.Table;
+import org.apache.calcite.schema.TableMacro;
+import org.apache.calcite.schema.TranslatableTable;
+import org.apache.calcite.schema.impl.AbstractSchema;
+import org.apache.calcite.schema.impl.AbstractTableQueryable;
+import org.apache.calcite.schema.impl.ReflectiveFunctionBase;
+import org.apache.calcite.util.BuiltInMethod;
 
-import net.hydromatic.optiq.*;
-import net.hydromatic.optiq.Table;
-import net.hydromatic.optiq.impl.*;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableMultimap;
+import com.google.common.collect.Multimap;
 
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.reltype.RelDataTypeFactory;
-
-import com.google.common.collect.*;
-
-import java.lang.reflect.*;
-import java.util.*;
+import java.lang.reflect.Field;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.lang.reflect.Type;
+import java.util.List;
+import java.util.Map;
 
 /**
- * Implementation of {@link net.hydromatic.optiq.Schema} that exposes the public
- * fields and methods in a Java object.
+ * Implementation of {@link org.apache.calcite.schema.Schema} that exposes the
+ * public fields and methods in a Java object.
  */
 public class ReflectiveSchema
     extends AbstractSchema {
@@ -51,19 +73,20 @@ public class ReflectiveSchema
     this.target = target;
   }
 
-  @Override
-  public String toString() {
+  @Override public String toString() {
     return "ReflectiveSchema(target=" + target + ")";
   }
 
-  /** Returns the wrapped object. (May not appear to be used, but is used in
-   * generated code via {@link BuiltinMethod#REFLECTIVE_SCHEMA_GET_TARGET}.) */
+  /** Returns the wrapped object.
+   *
+   * <p>May not appear to be used, but is used in generated code via
+   * {@link org.apache.calcite.util.BuiltInMethod#REFLECTIVE_SCHEMA_GET_TARGET}.
+   */
   public Object getTarget() {
     return target;
   }
 
-  @Override
-  protected Map<String, Table> getTableMap() {
+  @Override protected Map<String, Table> getTableMap() {
     final ImmutableMap.Builder<String, Table> builder = ImmutableMap.builder();
     for (Field field : clazz.getFields()) {
       final String fieldName = field.getName();
@@ -103,7 +126,7 @@ public class ReflectiveSchema
             Schemas.unwrap(
                 getExpression(parentSchema, name),
                 ReflectiveSchema.class),
-            BuiltinMethod.REFLECTIVE_SCHEMA_GET_TARGET.method));
+            BuiltInMethod.REFLECTIVE_SCHEMA_GET_TARGET.method));
   }
 
   /** Returns a table based on a particular field of this schema. If the
@@ -197,7 +220,7 @@ public class ReflectiveSchema
    *     {
    *       name: "foodmart",
    *       type: "custom",
-   *       factory: "net.hydromatic.optiq.impl.java.ReflectiveSchema$Factory",
+   *       factory: "org.apache.calcite.adapter.java.ReflectiveSchema$Factory",
    *       operand: {
    *         class: "com.acme.FoodMart",
    *         staticMethod: "instance"
@@ -261,7 +284,7 @@ public class ReflectiveSchema
       this.schema = schema;
       assert TranslatableTable.class.isAssignableFrom(method.getReturnType())
           : "Method should return TranslatableTable so the macro can be "
-            + "expanded";
+          + "expanded";
     }
 
     public String toString() {
@@ -293,9 +316,8 @@ public class ReflectiveSchema
       return "Relation {field=" + field.getName() + "}";
     }
 
-    @Override
-    public Expression getExpression(SchemaPlus schema, String tableName,
-        Class clazz) {
+    @Override public Expression getExpression(SchemaPlus schema,
+        String tableName, Class clazz) {
       return Expressions.field(
           schema.unwrap(ReflectiveSchema.class).getTargetExpression(
               schema.getParentSchema(), schema.getName()), field);

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/java/package-info.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/java/package-info.java b/core/src/main/java/org/apache/calcite/adapter/java/package-info.java
index c164afe..4d66ffe 100644
--- a/core/src/main/java/org/apache/calcite/adapter/java/package-info.java
+++ b/core/src/main/java/org/apache/calcite/adapter/java/package-info.java
@@ -19,6 +19,6 @@
  * Query provider based on Java in-memory data
  * structures.
  */
-package net.hydromatic.optiq.impl.java;
+package org.apache.calcite.adapter.java;
 
 // End package-info.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcConvention.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcConvention.java b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcConvention.java
index c28c363..1d2358b 100644
--- a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcConvention.java
+++ b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcConvention.java
@@ -14,16 +14,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl.jdbc;
+package org.apache.calcite.adapter.jdbc;
 
-import net.hydromatic.linq4j.expressions.Expression;
-
-import org.eigenbase.rel.rules.PushFilterPastSetOpRule;
-import org.eigenbase.rel.rules.RemoveTrivialProjectRule;
-import org.eigenbase.relopt.Convention;
-import org.eigenbase.relopt.RelOptPlanner;
-import org.eigenbase.relopt.RelOptRule;
-import org.eigenbase.sql.SqlDialect;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.rel.rules.FilterSetOpTransposeRule;
+import org.apache.calcite.rel.rules.ProjectRemoveRule;
+import org.apache.calcite.sql.SqlDialect;
 
 /**
  * Calling convention for relational operations that occur in a JDBC
@@ -59,13 +58,12 @@ public class JdbcConvention extends Convention.Impl {
     return new JdbcConvention(dialect, expression, name);
   }
 
-  @Override
-  public void register(RelOptPlanner planner) {
+  @Override public void register(RelOptPlanner planner) {
     for (RelOptRule rule : JdbcRules.rules(this)) {
       planner.addRule(rule);
     }
-    planner.addRule(PushFilterPastSetOpRule.INSTANCE);
-    planner.addRule(RemoveTrivialProjectRule.INSTANCE);
+    planner.addRule(FilterSetOpTransposeRule.INSTANCE);
+    planner.addRule(ProjectRemoveRule.INSTANCE);
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcImplementor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcImplementor.java b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcImplementor.java
index 89ae40f..7955a3a 100644
--- a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcImplementor.java
+++ b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcImplementor.java
@@ -14,30 +14,55 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl.jdbc;
-
-import net.hydromatic.linq4j.expressions.Expressions;
-
-import net.hydromatic.optiq.impl.java.JavaTypeFactory;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.reltype.RelDataTypeField;
-import org.eigenbase.rex.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.fun.SqlCase;
-import org.eigenbase.sql.fun.SqlStdOperatorTable;
-import org.eigenbase.sql.fun.SqlSumEmptyIsZeroAggFunction;
-import org.eigenbase.sql.parser.SqlParserPos;
-import org.eigenbase.sql.type.BasicSqlType;
-import org.eigenbase.sql.type.SqlTypeName;
-import org.eigenbase.sql.validate.SqlValidatorUtil;
-import org.eigenbase.util.Pair;
-import org.eigenbase.util.Util;
+package org.apache.calcite.adapter.jdbc;
+
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexLocalRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.sql.SqlAggFunction;
+import org.apache.calcite.sql.SqlBinaryOperator;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlDataTypeSpec;
+import org.apache.calcite.sql.SqlDialect;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlJoin;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlNodeList;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlSelect;
+import org.apache.calcite.sql.SqlSelectKeyword;
+import org.apache.calcite.sql.SqlSetOperator;
+import org.apache.calcite.sql.fun.SqlCase;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.fun.SqlSumEmptyIsZeroAggFunction;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.BasicSqlType;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.validate.SqlValidatorUtil;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
 
-import java.util.*;
+import java.util.AbstractList;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Set;
 
 /**
  * State for generating a SQL statement.
@@ -356,9 +381,9 @@ public class JdbcImplementor {
      * start a new SELECT that wraps the previous result.</p>
      *
      * <p>When you have called
-     * {@link Builder#setSelect(org.eigenbase.sql.SqlNodeList)},
-     * {@link Builder#setWhere(org.eigenbase.sql.SqlNode)} etc. call
-     * {@link Builder#result(org.eigenbase.sql.SqlNode, java.util.Collection, org.eigenbase.rel.RelNode)}
+     * {@link Builder#setSelect(org.apache.calcite.sql.SqlNodeList)},
+     * {@link Builder#setWhere(org.apache.calcite.sql.SqlNode)} etc. call
+     * {@link Builder#result(org.apache.calcite.sql.SqlNode, java.util.Collection, org.apache.calcite.rel.RelNode)}
      * to fix the new query.</p>
      *
      * @param rel Relational expression being implemented
@@ -387,8 +412,7 @@ public class JdbcImplementor {
       final SqlNodeList selectList = select.getSelectList();
       if (selectList != null) {
         newContext = new Context(selectList.size()) {
-          @Override
-          public SqlNode field(int ordinal) {
+          @Override public SqlNode field(int ordinal) {
             final SqlNode selectItem = selectList.get(ordinal);
             switch (selectItem.getKind()) {
             case AS:


[10/58] [abbrv] [partial] incubator-calcite git commit: [CALCITE-306] Standardize code style for "import package.*; "

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/test/SqlValidatorTestCase.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/SqlValidatorTestCase.java b/core/src/test/java/org/apache/calcite/test/SqlValidatorTestCase.java
index 80b166d..42b9b11 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlValidatorTestCase.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlValidatorTestCase.java
@@ -14,21 +14,28 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.test;
-
-import java.nio.charset.*;
-import java.util.regex.*;
-
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.sql.test.DefaultSqlTestFactory;
-import org.eigenbase.sql.test.SqlTester;
-import org.eigenbase.sql.test.SqlTesterImpl;
-import org.eigenbase.sql.validate.*;
-import org.eigenbase.util.*;
-
-import static org.junit.Assert.*;
+package org.apache.calcite.test;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.runtime.CalciteContextException;
+import org.apache.calcite.sql.SqlCollation;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.parser.SqlParseException;
+import org.apache.calcite.sql.parser.SqlParserUtil;
+import org.apache.calcite.sql.test.DefaultSqlTestFactory;
+import org.apache.calcite.sql.test.SqlTester;
+import org.apache.calcite.sql.test.SqlTesterImpl;
+import org.apache.calcite.sql.validate.SqlConformance;
+import org.apache.calcite.sql.validate.SqlMonotonicity;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.util.TestUtil;
+import org.apache.calcite.util.Util;
+
+import java.nio.charset.Charset;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import static org.junit.Assert.fail;
 
 /**
  * An abstract base class for implementing tests against {@link SqlValidator}.
@@ -210,9 +217,8 @@ public class SqlValidatorTestCase {
         // No error expected, and no error happened.
         return;
       } else {
-        throw new AssertionError(
-            "Expected query to throw exception, but it did not; "
-            + "query [" + sap.sql
+        throw new AssertionError("Expected query to throw exception, "
+            + "but it did not; query [" + sap.sql
             + "]; expected [" + expectedMsgPattern + "]");
       }
     }
@@ -223,11 +229,11 @@ public class SqlValidatorTestCase {
     int actualEndLine = 100;
     int actualEndColumn = 99;
 
-    // Search for an EigenbaseContextException somewhere in the stack.
-    EigenbaseContextException ece = null;
+    // Search for an CalciteContextException somewhere in the stack.
+    CalciteContextException ece = null;
     for (Throwable x = ex; x != null; x = x.getCause()) {
-      if (x instanceof EigenbaseContextException) {
-        ece = (EigenbaseContextException) x;
+      if (x instanceof CalciteContextException) {
+        ece = (CalciteContextException) x;
         break;
       }
       if (x.getCause() == x) {
@@ -313,8 +319,7 @@ public class SqlValidatorTestCase {
             || (actualEndColumn <= 0)
             || (actualEndLine <= 0)) {
           if (sap.pos != null) {
-            throw new AssertionError(
-                "Expected error to have position,"
+            throw new AssertionError("Expected error to have position,"
                 + " but actual error did not: "
                 + " actual pos [line " + actualLine
                 + " col " + actualColumn
@@ -347,8 +352,7 @@ public class SqlValidatorTestCase {
               (actualMessage == null) ? "null"
                   : TestUtil.quoteForJava(
                       TestUtil.quotePattern(actualMessage));
-          fail(
-              "Validator threw different "
+          fail("Validator threw different "
               + "exception than expected; query [" + sap.sql
               + "];\n"
               + " expected pattern [" + expectedMsgPattern

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/test/TableInRootSchemaTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/TableInRootSchemaTest.java b/core/src/test/java/org/apache/calcite/test/TableInRootSchemaTest.java
index 53d0362..58ebe99 100644
--- a/core/src/test/java/org/apache/calcite/test/TableInRootSchemaTest.java
+++ b/core/src/test/java/org/apache/calcite/test/TableInRootSchemaTest.java
@@ -14,43 +14,56 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.test;
-
-import net.hydromatic.linq4j.*;
-
-import net.hydromatic.optiq.*;
-import net.hydromatic.optiq.impl.AbstractTableQueryable;
-import net.hydromatic.optiq.impl.java.AbstractQueryableTable;
-import net.hydromatic.optiq.jdbc.OptiqConnection;
-import net.hydromatic.optiq.rules.java.EnumerableConvention;
-import net.hydromatic.optiq.rules.java.JavaRules;
-
-import org.eigenbase.rel.RelNode;
-import org.eigenbase.relopt.RelOptTable;
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.reltype.RelDataTypeFactory;
-import org.eigenbase.util.Pair;
+package org.apache.calcite.test;
+
+import org.apache.calcite.adapter.enumerable.EnumerableConvention;
+import org.apache.calcite.adapter.enumerable.EnumerableTableScan;
+import org.apache.calcite.adapter.java.AbstractQueryableTable;
+import org.apache.calcite.jdbc.CalciteConnection;
+import org.apache.calcite.linq4j.Enumerator;
+import org.apache.calcite.linq4j.Linq4j;
+import org.apache.calcite.linq4j.QueryProvider;
+import org.apache.calcite.linq4j.Queryable;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.schema.TranslatableTable;
+import org.apache.calcite.schema.impl.AbstractTableQueryable;
+import org.apache.calcite.util.Pair;
 
 import com.google.common.collect.ImmutableMultiset;
 
 import org.junit.Test;
 
-import java.sql.*;
-import java.util.*;
-
-import static org.hamcrest.CoreMatchers.*;
-import static org.junit.Assert.*;
-
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.Statement;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
+
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.CoreMatchers.nullValue;
+import static org.junit.Assert.assertThat;
+
+/** Test case for issue 85. */
 public class TableInRootSchemaTest {
-  /** Test case for issue 85, "Adding a table to the root schema causes breakage
-   * in OptiqPrepareImpl". */
+  /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-85">[CALCITE-85]
+   * Adding a table to the root schema causes breakage in
+   * CalcitePrepareImpl</a>. */
   @Test public void testAddingTableInRootSchema() throws Exception {
-    Class.forName("net.hydromatic.optiq.jdbc.Driver");
     Connection connection = DriverManager.getConnection("jdbc:calcite:");
-    OptiqConnection optiqConnection = connection.unwrap(OptiqConnection.class);
+    CalciteConnection calciteConnection =
+        connection.unwrap(CalciteConnection.class);
 
-    optiqConnection.getRootSchema().add("SAMPLE", new SimpleTable());
-    Statement statement = optiqConnection.createStatement();
+    calciteConnection.getRootSchema().add("SAMPLE", new SimpleTable());
+    Statement statement = calciteConnection.createStatement();
     ResultSet resultSet =
         statement.executeQuery("select A, SUM(B) from SAMPLE group by A");
 
@@ -58,7 +71,7 @@ public class TableInRootSchemaTest {
         ImmutableMultiset.of(
             "A=foo; EXPR$1=8",
             "A=bar; EXPR$1=4"),
-        equalTo(OptiqAssert.toSet(resultSet)));
+        equalTo(CalciteAssert.toSet(resultSet)));
 
     final ResultSetMetaData resultSetMetaData = resultSet.getMetaData();
     assertThat(resultSetMetaData.getColumnName(1), equalTo("A"));
@@ -74,6 +87,7 @@ public class TableInRootSchemaTest {
     connection.close();
   }
 
+  /** Table with columns (A, B). */
   public static class SimpleTable extends AbstractQueryableTable
       implements TranslatableTable {
     private String[] columnNames = { "A", "B" };
@@ -161,7 +175,7 @@ public class TableInRootSchemaTest {
     // keep
     public RelNode toRel(RelOptTable.ToRelContext context,
         RelOptTable relOptTable) {
-      return new JavaRules.EnumerableTableAccessRel(context.getCluster(),
+      return new EnumerableTableScan(context.getCluster(),
           context.getCluster().traitSetOf(EnumerableConvention.INSTANCE),
           relOptTable, (Class) getElementType());
     }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/test/concurrent/ConcurrentTestCommand.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/concurrent/ConcurrentTestCommand.java b/core/src/test/java/org/apache/calcite/test/concurrent/ConcurrentTestCommand.java
index 3a735be..5915ee9 100644
--- a/core/src/test/java/org/apache/calcite/test/concurrent/ConcurrentTestCommand.java
+++ b/core/src/test/java/org/apache/calcite/test/concurrent/ConcurrentTestCommand.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.test.concurrent;
+package org.apache.calcite.test.concurrent;
 
 /**
  * ConcurrentTestCommand represents a command, sequentially executed by

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/test/concurrent/ConcurrentTestCommandExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/concurrent/ConcurrentTestCommandExecutor.java b/core/src/test/java/org/apache/calcite/test/concurrent/ConcurrentTestCommandExecutor.java
index 8b77b86..25d89eb 100644
--- a/core/src/test/java/org/apache/calcite/test/concurrent/ConcurrentTestCommandExecutor.java
+++ b/core/src/test/java/org/apache/calcite/test/concurrent/ConcurrentTestCommandExecutor.java
@@ -14,11 +14,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.test.concurrent;
+package org.apache.calcite.test.concurrent;
 
 import java.io.PrintStream;
-import java.sql.*;
-import java.util.*;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.Statement;
+import java.util.Properties;
 
 /**
  * ConcurrentTestCommandExecutor is a thread that executes a sequence of
@@ -88,7 +90,8 @@ class ConcurrentTestCommandExecutor extends Thread {
    * Constructs a ConcurrentTestCommandExecutor with the given thread
    * ID, JDBC URL, commands and synchronization object.
    *
-   * @param threadId         the thread ID (see {@link ConcurrentTestCommandGenerator})
+   * @param threadId         the thread ID
+   *                         (see {@link ConcurrentTestCommandGenerator})
    * @param threadName       the thread's name
    * @param jdbcURL          the JDBC URL to connect to
    * @param jdbcProps        JDBC Connection properties (user, password, etc.)
@@ -249,8 +252,8 @@ class ConcurrentTestCommandExecutor extends Thread {
   }
 
   /**
-   * Returns location (e.g., command number) for exception returned by {@link
-   * #getFailureCause()}.
+   * Returns location (e.g., command number) for exception returned by
+   * {@link #getFailureCause()}.
    */
   public String getFailureLocation() {
     return when;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/test/concurrent/ConcurrentTestCommandGenerator.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/concurrent/ConcurrentTestCommandGenerator.java b/core/src/test/java/org/apache/calcite/test/concurrent/ConcurrentTestCommandGenerator.java
index 0709de3..b9e332d 100644
--- a/core/src/test/java/org/apache/calcite/test/concurrent/ConcurrentTestCommandGenerator.java
+++ b/core/src/test/java/org/apache/calcite/test/concurrent/ConcurrentTestCommandGenerator.java
@@ -14,25 +14,36 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.test.concurrent;
+package org.apache.calcite.test.concurrent;
 
-import java.io.PrintStream;
-import java.math.*;
-import java.sql.*;
-import java.util.*;
-import java.util.regex.Pattern;
-
-import org.eigenbase.util.Util;
-
-import net.hydromatic.optiq.jdbc.SqlTimeoutException;
+import org.apache.calcite.jdbc.SqlTimeoutException;
+import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
 
+import java.io.PrintStream;
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.regex.Pattern;
+
 /**
- * ConcurrentTestCommandGenerator creates instances of {@link
- * ConcurrentTestCommand} that perform specific actions in a specific
- * order and within the context of a test thread ({@link
- * ConcurrentTestCommandExecutor}).
+ * ConcurrentTestCommandGenerator creates instances of
+ * {@link ConcurrentTestCommand} that perform specific actions in a specific
+ * order and within the context of a test thread
+ * ({@link ConcurrentTestCommandExecutor}).
  *
  * <p>Typical actions include preparing a SQL statement for execution, executing
  * the statement and verifying its result set, and closing the statement.</p>
@@ -197,9 +208,9 @@ public class ConcurrentTestCommandGenerator {
   }
 
   /**
-   * Executes a previously {@link #addPrepareCommand(int, int, String)
-   * prepared} SQL statement and compares its {@link ResultSet} to the given
-   * data.
+   * Executes a previously
+   * {@link #addPrepareCommand(int, int, String) prepared} SQL statement and
+   * compares its {@link ResultSet} to the given data.
    *
    * <p><b>Expected data format:</b> <code>{ 'row1, col1 value', 'row1, col2
    * value', ... }, { 'row2, col1 value', 'row2, col2 value', ... },
@@ -381,8 +392,8 @@ public class ConcurrentTestCommandGenerator {
 
 
   /**
-   * Creates a {@link ConcurrentTestCommandExecutor} object for each define thread,
-   * and then runs them all.
+   * Creates a {@link ConcurrentTestCommandExecutor} object for each define
+   * thread, and then runs them all.
    *
    * @throws Exception if no connection found or if a thread operation is
    *                   interrupted
@@ -407,8 +418,7 @@ public class ConcurrentTestCommandGenerator {
       Iterable<ConcurrentTestCommand> commands = getCommandIterable(threadId);
 
       if (debug) {
-        debugStream.println(
-            "Thread ID: " + threadId + " ("
+        debugStream.println("Thread ID: " + threadId + " ("
             + getThreadName(threadId)
             + ")");
         printCommands(debugStream, threadId);
@@ -585,9 +595,9 @@ public class ConcurrentTestCommandGenerator {
   }
 
   /**
-   * Custom error handling occurs here if {@link
-   * #requiresCustomErrorHandling()} returns true. Default implementation does
-   * nothing.
+   * Custom error handling occurs here if
+   * {@link #requiresCustomErrorHandling()} returns true. Default implementation
+   * does nothing.
    */
   void customErrorHandler(
       ConcurrentTestCommandExecutor executor) {
@@ -743,8 +753,8 @@ public class ConcurrentTestCommandGenerator {
 
   /**
    * ExplainCommand executes explain plan commands. Automatically closes the
-   * {@link Statement} before returning from {@link
-   * #execute(ConcurrentTestCommandExecutor)}.
+   * {@link Statement} before returning from
+   * {@link #execute(ConcurrentTestCommandExecutor)}.
    */
   private static class ExplainCommand extends AbstractCommand {
     private String sql;
@@ -815,6 +825,7 @@ public class ConcurrentTestCommandGenerator {
     }
   }
 
+  /** Command that executes statements with a given timeout. */
   private abstract static class CommandWithTimeout extends AbstractCommand {
     private int timeout;
 
@@ -899,9 +910,9 @@ public class ConcurrentTestCommandGenerator {
     private static final int STATE_VALUE_END = 4;
 
     /**
-     * Parses expected values. See {@link
-     * ConcurrentTestCommandGenerator#addFetchAndCompareCommand(int,
-     * int, int, String)} for details on format of <code>expected</code>.
+     * Parses expected values. See
+     * {@link ConcurrentTestCommandGenerator#addFetchAndCompareCommand(int, int, int, String)}
+     * for details on format of <code>expected</code>.
      *
      * @throws IllegalStateException if there are formatting errors in
      *                               <code>expected</code>
@@ -1311,8 +1322,8 @@ public class ConcurrentTestCommandGenerator {
   }
 
   /**
-   * CommitCommand commits pending transactions via {@link
-   * Connection#commit()}.
+   * CommitCommand commits pending transactions via
+   * {@link Connection#commit()}.
    */
   private static class CommitCommand extends AbstractCommand {
     protected void doExecute(ConcurrentTestCommandExecutor executor)
@@ -1322,8 +1333,8 @@ public class ConcurrentTestCommandGenerator {
   }
 
   /**
-   * RollbackCommand rolls back pending transactions via {@link
-   * Connection#rollback()}.
+   * RollbackCommand rolls back pending transactions via
+   * {@link Connection#rollback()}.
    */
   private static class RollbackCommand extends AbstractCommand {
     protected void doExecute(ConcurrentTestCommandExecutor executor)
@@ -1333,9 +1344,9 @@ public class ConcurrentTestCommandGenerator {
   }
 
   /**
-   * DdlCommand executes DDL commands. Automatically closes the {@link
-   * Statement} before returning from {@link
-   * #doExecute(ConcurrentTestCommandExecutor)}.
+   * DdlCommand executes DDL commands. Automatically closes the
+   * {@link Statement} before returning from
+   * {@link #doExecute(ConcurrentTestCommandExecutor)}.
    */
   private static class DdlCommand extends AbstractCommand {
     private String sql;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/test/concurrent/ConcurrentTestCommandScript.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/concurrent/ConcurrentTestCommandScript.java b/core/src/test/java/org/apache/calcite/test/concurrent/ConcurrentTestCommandScript.java
index 162c480..8c82119 100644
--- a/core/src/test/java/org/apache/calcite/test/concurrent/ConcurrentTestCommandScript.java
+++ b/core/src/test/java/org/apache/calcite/test/concurrent/ConcurrentTestCommandScript.java
@@ -14,22 +14,46 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.test.concurrent;
-
-import java.io.*;
-import java.sql.*;
-import java.util.*;
-import java.util.regex.*;
-
-import org.eigenbase.util.Util;
-
-import net.hydromatic.optiq.jdbc.SqlTimeoutException;
+package org.apache.calcite.test.concurrent;
+
+import org.apache.calcite.jdbc.SqlTimeoutException;
+import org.apache.calcite.util.Util;
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.io.PrintWriter;
+import java.io.StringReader;
+import java.io.StringWriter;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.sql.Timestamp;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Stack;
+import java.util.StringTokenizer;
+import java.util.TreeMap;
+import java.util.TreeSet;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
 
 /**
- * ConcurrentTestCommandScript creates instances of {@link
- * ConcurrentTestCommand} that perform specific actions in a specific
- * order and within the context of a test thread ({@link
- * ConcurrentTestCommandExecutor}).
+ * ConcurrentTestCommandScript creates instances of
+ * {@link ConcurrentTestCommand} that perform specific actions in a specific
+ * order and within the context of a test thread
+ * ({@link ConcurrentTestCommandExecutor}).
  *
  * <p>Actions are loaded from a script (see package javadoc for script format).
  *
@@ -333,7 +357,7 @@ public class ConcurrentTestCommandScript
     executeCommands(CLEANUP_THREAD_ID, cleanupCommands);
   }
 
-  protected void executeCommands(int threadID, List<String> commands)
+  protected void executeCommands(int threadId, List<String> commands)
       throws Exception {
     if (commands == null || commands.size() == 0) {
       return;
@@ -348,7 +372,7 @@ public class ConcurrentTestCommandScript
     try {
       for (String command : commands) {
         String sql = command.trim();
-        storeSql(threadID, sql);
+        storeSql(threadId, sql);
 
         if (isComment(sql)) {
           continue;
@@ -375,7 +399,7 @@ public class ConcurrentTestCommandScript
           Statement stmt = connection.createStatement();
           try {
             ResultSet rset = stmt.executeQuery(sql);
-            storeResults(threadID, rset, -1);
+            storeResults(threadId, rset, -1);
           } finally {
             stmt.close();
           }
@@ -388,16 +412,14 @@ public class ConcurrentTestCommandScript
           try {
             int rows = stmt.executeUpdate(sql);
             if (rows != 1) {
-              storeMessage(
-                  threadID,
-                  String.valueOf(rows)
-                      + " rows affected.");
+              storeMessage(threadId,
+                  String.valueOf(rows) + " rows affected.");
             } else {
-              storeMessage(threadID, "1 row affected.");
+              storeMessage(threadId, "1 row affected.");
             }
           } catch (SQLException ex) {
             if (forced) {
-              storeMessage(threadID, ex.getMessage()); // swallow
+              storeMessage(threadId, ex.getMessage()); // swallow
             } else {
               throw ex;
             }
@@ -546,8 +568,8 @@ public class ConcurrentTestCommandScript
   }
 
   /**
-   * Causes errors to be send here for custom handling. See {@link
-   * #customErrorHandler(ConcurrentTestCommandExecutor)}.
+   * Causes errors to be send here for custom handling. See
+   * {@link #customErrorHandler(ConcurrentTestCommandExecutor)}.
    */
   boolean requiresCustomErrorHandling() {
     return true;
@@ -640,6 +662,7 @@ public class ConcurrentTestCommandScript
 
   //~ Inner Classes ----------------------------------------------------------
 
+  /** State action. */
   private static class StateAction {
     final String state;
     final StateDatum[] stateData;
@@ -650,6 +673,7 @@ public class ConcurrentTestCommandScript
     }
   }
 
+  /** State datum. */
   private static class StateDatum {
     final String x;
     final String y;
@@ -661,8 +685,7 @@ public class ConcurrentTestCommandScript
   }
 
 
-  // Inner Class: a symbol table of script variables
-
+  /** Symbol table of script variables. */
   private class VariableTable {
     private final Map<String, String> map;
 
@@ -674,6 +697,7 @@ public class ConcurrentTestCommandScript
       map = new HashMap<String, String>();
     }
 
+    /** Exception. */
     public class Excn extends IllegalArgumentException {
       public Excn(String msg) {
         super(msg);
@@ -751,8 +775,7 @@ public class ConcurrentTestCommandScript
   }
 
 
-  // Inner Class: the command parser
-
+  /** Command parser. */
   private class CommandParser {
     final Pattern splitWords = Pattern.compile("\\s+");
     final Pattern splitBinding = Pattern.compile("=");
@@ -769,6 +792,7 @@ public class ConcurrentTestCommandScript
     private boolean scriptHasVars;
     private Stack<File> currentDirectory = new Stack<File>();
 
+    /** Binding of a value to a variable. */
     private class Binding {
       public final String var;
       public final String val;
@@ -1335,13 +1359,14 @@ public class ConcurrentTestCommandScript
   }
 
 
-  // Inner Classes: the Commands
-
-  // When executed, a @print command defines how any following @fetch
-  // or @select commands will handle their resuult rows. MTSQL can print all
-  // rows, no rows, or every nth row. A printed row can be prefixed by a
-  // sequence nuber and/or the time it was received (a different notion than
-  // its rowtime, which often tells when it was inserted).
+  /** Print command.
+   *
+   * <p>When executed, a @print command defines how any following @fetch
+   * or @select commands will handle their resuult rows. MTSQL can print all
+   * rows, no rows, or every nth row. A printed row can be prefixed by a
+   * sequence nuber and/or the time it was received (a different notion than
+   * its rowtime, which often tells when it was inserted).
+   */
   private class PrintCommand extends AbstractCommand {
     // print every nth row: 1 means all rows, 0 means no rows.
     private final int nth;
@@ -1398,6 +1423,7 @@ public class ConcurrentTestCommandScript
     }
   }
 
+  /** Echo command. */
   private class EchoCommand extends AbstractCommand {
     private final String msg;
 
@@ -1411,6 +1437,7 @@ public class ConcurrentTestCommandScript
     }
   }
 
+  /** Plugin command. */
   private class PluginCommand extends AbstractCommand {
 
     private final ConcurrentTestPluginCommand pluginCommand;
@@ -1448,6 +1475,7 @@ public class ConcurrentTestCommandScript
   private final Pattern shellWildcardPattern = Pattern.compile("[*?$|<>&]");
 
   // REVIEW mb 2/24/09 (Mardi Gras) Should this have a timeout?
+  /** Shell command. */
   private class ShellCommand extends AbstractCommand {
     private final String command;
     private List<String> argv;      // the command, parsed and massaged
@@ -1492,19 +1520,17 @@ public class ConcurrentTestCommandScript
             Util.runAppProcess(
                 pb, null, null, getThreadWriter(threadId));
         if (status != 0) {
-          storeMessage(
-              threadId, "command " + command
-              + ": exited with status " + status);
+          storeMessage(threadId,
+              "command " + command + ": exited with status " + status);
         }
       } catch (Exception e) {
-        storeMessage(
-            threadId, "command " + command
-            + ": failed with exception " + e.getMessage());
+        storeMessage(threadId,
+            "command " + command + ": failed with exception " + e.getMessage());
       }
     }
   }
 
-
+  /** Command that has a timeout. */
   // TODO: replace by super.CommmandWithTimeout
   private abstract static class CommandWithTimeout extends AbstractCommand {
     private long timeout;
@@ -1527,6 +1553,7 @@ public class ConcurrentTestCommandScript
     }
   }
 
+  /** Command with timeout and row limit. */
   private abstract static class CommandWithTimeoutAndRowLimit
       extends CommandWithTimeout {
     private int rowLimit;
@@ -1749,6 +1776,7 @@ public class ConcurrentTestCommandScript
     }
   }
 
+  /** Result reader. */
   private class ResultsReader {
     private final PrintWriter out;
     // print every Nth row. 1 means all rows, 0 means none.
@@ -1972,7 +2000,7 @@ public class ConcurrentTestCommandScript
   }
 
 
-  // Inner class: stand-alone client test tool
+  /** Standalone client test tool. */
   private static class Tool {
     boolean quiet = false;          // -q
     boolean verbose = false;        // -v

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/test/concurrent/ConcurrentTestPlugin.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/concurrent/ConcurrentTestPlugin.java b/core/src/test/java/org/apache/calcite/test/concurrent/ConcurrentTestPlugin.java
index 3741507..290d795 100644
--- a/core/src/test/java/org/apache/calcite/test/concurrent/ConcurrentTestPlugin.java
+++ b/core/src/test/java/org/apache/calcite/test/concurrent/ConcurrentTestPlugin.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.test.concurrent;
+package org.apache.calcite.test.concurrent;
 
 import java.util.ArrayList;
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/test/concurrent/ConcurrentTestPluginCommand.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/concurrent/ConcurrentTestPluginCommand.java b/core/src/test/java/org/apache/calcite/test/concurrent/ConcurrentTestPluginCommand.java
index 5432feb..d5a2335 100644
--- a/core/src/test/java/org/apache/calcite/test/concurrent/ConcurrentTestPluginCommand.java
+++ b/core/src/test/java/org/apache/calcite/test/concurrent/ConcurrentTestPluginCommand.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.test.concurrent;
+package org.apache.calcite.test.concurrent;
 
 import java.io.IOException;
 import java.sql.Connection;
@@ -25,6 +25,7 @@ import java.sql.Statement;
  */
 public interface ConcurrentTestPluginCommand {
 
+  /** Test context. */
   interface TestContext {
     /**
      * Store a message as output for mtsql script.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/test/concurrent/ConcurrentTestTimedCommandGenerator.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/concurrent/ConcurrentTestTimedCommandGenerator.java b/core/src/test/java/org/apache/calcite/test/concurrent/ConcurrentTestTimedCommandGenerator.java
index eeb8147..15e177c 100644
--- a/core/src/test/java/org/apache/calcite/test/concurrent/ConcurrentTestTimedCommandGenerator.java
+++ b/core/src/test/java/org/apache/calcite/test/concurrent/ConcurrentTestTimedCommandGenerator.java
@@ -14,16 +14,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.test.concurrent;
-
-import java.io.PrintStream;
-import java.util.*;
+package org.apache.calcite.test.concurrent;
 
 import com.google.common.collect.ImmutableList;
 
+import java.io.PrintStream;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+
 /**
- * ConcurrentTestTimedCommandGenerator extends {@link
- * ConcurrentTestCommandGenerator} and repeats the configured command
+ * ConcurrentTestTimedCommandGenerator extends
+ * {@link ConcurrentTestCommandGenerator} and repeats the configured command
  * sequence until a certain amount of time has elapsed.
  *
  * <p>The command sequence is always completed in full, even if the time limit
@@ -42,8 +45,8 @@ public class ConcurrentTestTimedCommandGenerator
 
   /**
    * Constructs a new ConcurrentTestTimedCommandGenerator that will run
-   * for at least the given amount of time. See {@link
-   * ConcurrentTestTimedCommandGenerator} for more information on the
+   * for at least the given amount of time. See
+   * {@link ConcurrentTestTimedCommandGenerator} for more information on the
    * semantics of run-time length.
    *
    * @param runTimeSeconds minimum run-time length, in seconds

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/test/concurrent/SamplePlugin.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/concurrent/SamplePlugin.java b/core/src/test/java/org/apache/calcite/test/concurrent/SamplePlugin.java
index 54089b4..3630e48 100644
--- a/core/src/test/java/org/apache/calcite/test/concurrent/SamplePlugin.java
+++ b/core/src/test/java/org/apache/calcite/test/concurrent/SamplePlugin.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.test.concurrent;
+package org.apache.calcite.test.concurrent;
 
 import java.io.IOException;
 import java.sql.PreparedStatement;
@@ -24,9 +24,11 @@ import java.sql.Statement;
 import java.util.Arrays;
 
 /**
- * Sample mtsql plugin. To use add at start of script
- * "@plugin org.eigenbase.test.concurrent.SamplePlugin".  After doing a prepare
- * you can then do "@describeResultSet" to show columns returned by query.
+ * Sample mtsql plugin.
+ *
+ * <p>To use add at start of script "@plugin
+ * org.apache.calcite.test.concurrent.SamplePlugin".  After doing a prepare you
+ * can then do "@describeResultSet" to show columns returned by query.
  */
 public class SamplePlugin extends ConcurrentTestPlugin {
   private static final String DESCRIBE_RESULT_SET_CMD = "@describeResultSet";
@@ -43,8 +45,8 @@ public class SamplePlugin extends ConcurrentTestPlugin {
     return Arrays.asList(new String[]{DESCRIBE_RESULT_SET_CMD});
   }
 
+  /** Command that describes a result set. */
   static class DescribeResultSet implements ConcurrentTestPluginCommand {
-
     public void execute(TestContext testContext) throws IOException {
       Statement stmt =
           (PreparedStatement) testContext.getCurrentStatement();

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/test/concurrent/package-info.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/concurrent/package-info.java b/core/src/test/java/org/apache/calcite/test/concurrent/package-info.java
index 6992a8b..38a59d0 100644
--- a/core/src/test/java/org/apache/calcite/test/concurrent/package-info.java
+++ b/core/src/test/java/org/apache/calcite/test/concurrent/package-info.java
@@ -37,8 +37,7 @@
  *
  * <h4>Syntax:</h4>
  *
- * <p>
- *     The syntactic structure of an <i>mtsql</i> script is:
+ * <p>The syntactic structure of an <i>mtsql</i> script is:
  *
  * <pre>
  *     &lt;directive&gt;*
@@ -58,8 +57,7 @@
  *       &lt;synchronization point&gt;
  * </pre>
  *
- * <p>
- *     Blank lines and comments are allowed anywhere.
+ * <p>Blank lines and comments are allowed anywhere.
  *     A comment starts with two hyphens and runs to the end of the line.
  *     Command names start with an '&#64;'.
  *     Some commands run to the end of the line; but a command that contains SQL
@@ -68,8 +66,7 @@
  *
  * <h4>Semantics:</h4>
  *
- * <p>
- *     Running a section means running its commands in sequence.
+ * <p>Running a section means running its commands in sequence.
  *     First the setup section (if any) is run.
  *     Next all the thread sections are run at once, each in its own thread.
  *     When all these threads complete, the cleanup section (if any) is run.
@@ -169,8 +166,7 @@
  *
  * <h4>Substituted Variables</h4>
  *
- * <p>
- *     Needed mainly to pass arguments to the command of &#64;shell, but also
+ * <p>Needed mainly to pass arguments to the command of &#64;shell, but also
  *     useful to
  *     parameterize SQL statements, timeout values etc.
  *
@@ -318,6 +314,6 @@
  *         </ul>
  *
  */
-package org.eigenbase.test.concurrent;
+package org.apache.calcite.test.concurrent;
 
 // End package-info.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/test/package-info.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/package-info.java b/core/src/test/java/org/apache/calcite/test/package-info.java
index fbf1f71..013b476 100644
--- a/core/src/test/java/org/apache/calcite/test/package-info.java
+++ b/core/src/test/java/org/apache/calcite/test/package-info.java
@@ -16,8 +16,8 @@
  */
 
 /**
- * Eigenbase regression tests.
+ * Tests for Calcite.
  */
-package org.eigenbase.test;
+package org.apache.calcite.test;
 
 // End package-info.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/tools/FrameworksTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/tools/FrameworksTest.java b/core/src/test/java/org/apache/calcite/tools/FrameworksTest.java
index 7188a5f..e882fc2 100644
--- a/core/src/test/java/org/apache/calcite/tools/FrameworksTest.java
+++ b/core/src/test/java/org/apache/calcite/tools/FrameworksTest.java
@@ -14,36 +14,41 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.tools;
-
-import net.hydromatic.optiq.SchemaPlus;
-import net.hydromatic.optiq.Table;
-import net.hydromatic.optiq.impl.AbstractTable;
-import net.hydromatic.optiq.rules.java.EnumerableConvention;
-import net.hydromatic.optiq.rules.java.JavaRules;
-import net.hydromatic.optiq.server.OptiqServerStatement;
-
-import org.eigenbase.rel.FilterRel;
-import org.eigenbase.rel.RelNode;
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.reltype.RelDataTypeFactory;
-import org.eigenbase.reltype.RelDataTypeSystem;
-import org.eigenbase.reltype.RelDataTypeSystemImpl;
-import org.eigenbase.rex.RexBuilder;
-import org.eigenbase.rex.RexLiteral;
-import org.eigenbase.rex.RexNode;
-import org.eigenbase.sql.SqlExplainLevel;
-import org.eigenbase.sql.fun.SqlStdOperatorTable;
-import org.eigenbase.sql.type.SqlTypeName;
-import org.eigenbase.util.Util;
+package org.apache.calcite.tools;
+
+import org.apache.calcite.adapter.enumerable.EnumerableConvention;
+import org.apache.calcite.adapter.enumerable.EnumerableTableScan;
+import org.apache.calcite.plan.RelOptAbstractTable;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelOptSchema;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.logical.LogicalFilter;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.rel.type.RelDataTypeSystemImpl;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.schema.Table;
+import org.apache.calcite.schema.impl.AbstractTable;
+import org.apache.calcite.server.CalciteServerStatement;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.util.Util;
 
 import org.junit.Test;
 
 import java.math.BigDecimal;
 
-import static org.hamcrest.CoreMatchers.*;
-import static org.junit.Assert.*;
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
 
 /**
  * Unit tests for methods in {@link Frameworks}.
@@ -75,8 +80,8 @@ public class FrameworksTest {
                 "myTable",
                 table.getRowType(typeFactory)) {
             };
-            final JavaRules.EnumerableTableAccessRel tableRel =
-                new JavaRules.EnumerableTableAccessRel(
+            final EnumerableTableScan tableRel =
+                new EnumerableTableScan(
                     cluster, cluster.traitSetOf(EnumerableConvention.INSTANCE),
                     relOptTable, Object[].class);
 
@@ -87,11 +92,11 @@ public class FrameworksTest {
                     rexBuilder.makeFieldAccess(
                         rexBuilder.makeRangeReference(tableRel), "i", true),
                     rexBuilder.makeExactLiteral(BigDecimal.ONE));
-            final FilterRel filterRel =
-                new FilterRel(cluster, tableRel, condition);
+            final LogicalFilter filter =
+                new LogicalFilter(cluster, tableRel, condition);
 
             // Specify that the result should be in Enumerable convention.
-            final RelNode rootRel = filterRel;
+            final RelNode rootRel = filter;
             final RelOptPlanner planner = cluster.getPlanner();
             RelTraitSet desiredTraits = rootRel.getTraitSet().replace(
                 EnumerableConvention.INSTANCE);
@@ -105,9 +110,9 @@ public class FrameworksTest {
         });
     String s =
         RelOptUtil.dumpPlan("", x, false, SqlExplainLevel.DIGEST_ATTRIBUTES);
-    assertThat(Util.toLinux(s), equalTo(
-        "EnumerableFilterRel(condition=[>($1, 1)])\n"
-        + "  EnumerableTableAccessRel(table=[[myTable]])\n"));
+    assertThat(Util.toLinux(s),
+        equalTo("EnumerableFilter(condition=[>($1, 1)])\n"
+            + "  EnumerableTableScan(table=[[myTable]])\n"));
   }
 
   /** Unit test to test create root schema which has no "metadata" schema. */
@@ -140,7 +145,7 @@ public class FrameworksTest {
         new Frameworks.PrepareAction<Void>(config) {
           @Override public Void apply(RelOptCluster cluster,
               RelOptSchema relOptSchema, SchemaPlus rootSchema,
-              OptiqServerStatement statement) {
+              CalciteServerStatement statement) {
             final RelDataType type =
                 cluster.getTypeFactory()
                     .createSqlType(SqlTypeName.DECIMAL, 30, 2);


[28/58] [abbrv] [partial] incubator-calcite git commit: [CALCITE-306] Standardize code style for "import package.*; "

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/schema/ScannableTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/schema/ScannableTable.java b/core/src/main/java/org/apache/calcite/schema/ScannableTable.java
index af76df9..908d745 100644
--- a/core/src/main/java/org/apache/calcite/schema/ScannableTable.java
+++ b/core/src/main/java/org/apache/calcite/schema/ScannableTable.java
@@ -14,9 +14,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq;
+package org.apache.calcite.schema;
 
-import net.hydromatic.linq4j.Enumerable;
+import org.apache.calcite.DataContext;
+import org.apache.calcite.linq4j.Enumerable;
 
 /**
  * Table that can be scanned without creating an intermediate relational

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/schema/Schema.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/schema/Schema.java b/core/src/main/java/org/apache/calcite/schema/Schema.java
index 1ed8743..0c46117 100644
--- a/core/src/main/java/org/apache/calcite/schema/Schema.java
+++ b/core/src/main/java/org/apache/calcite/schema/Schema.java
@@ -14,11 +14,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq;
+package org.apache.calcite.schema;
 
-import net.hydromatic.linq4j.expressions.Expression;
+import org.apache.calcite.linq4j.tree.Expression;
 
-import java.util.*;
+import java.util.Collection;
+import java.util.Set;
 
 /**
  * A namespace for tables and functions.
@@ -32,7 +33,7 @@ import java.util.*;
  * different numbers or types of parameters.
  * For this reason, {@link #getFunctions} returns a list of all
  * members with the same name. Calcite will call
- * {@link Schemas#resolve(org.eigenbase.reltype.RelDataTypeFactory, String, java.util.Collection, java.util.List)}
+ * {@link Schemas#resolve(org.apache.calcite.rel.type.RelDataTypeFactory, String, java.util.Collection, java.util.List)}
  * to choose the appropriate one.</p>
  *
  * <p>The most common and important type of member is the one with no
@@ -112,7 +113,7 @@ public interface Schema {
    * caching is enabled, Calcite will re-build caches.
    *
    * <p>The default implementation in
-   * {@link net.hydromatic.optiq.impl.AbstractSchema} always returns
+   * {@link org.apache.calcite.schema.impl.AbstractSchema} always returns
    * {@code false}.</p>
    *
    * <p>To control whether Calcite caches the contents of a schema, use the

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/schema/SchemaFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/schema/SchemaFactory.java b/core/src/main/java/org/apache/calcite/schema/SchemaFactory.java
index 34622b5..b2a78b2 100644
--- a/core/src/main/java/org/apache/calcite/schema/SchemaFactory.java
+++ b/core/src/main/java/org/apache/calcite/schema/SchemaFactory.java
@@ -14,12 +14,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq;
+package org.apache.calcite.schema;
 
 import java.util.Map;
 
 /**
- * Factory for {@link net.hydromatic.optiq.Schema} objects.
+ * Factory for {@link Schema} objects.
  *
  * <p>A schema factory allows you to include a custom schema in a model file.
  * For example, here is a model that contains a custom schema whose tables
@@ -35,7 +35,7 @@ import java.util.Map;
  *     {
  *       name: 'SALES',
  *       type: 'custom',
- *       factory: 'net.hydromatic.optiq.impl.csv.CsvSchemaFactory',
+ *       factory: 'org.apache.calcite.adapter.csv.CsvSchemaFactory',
  *       mutable: true,
  *       operand: {
  *         directory: 'target/test-classes/sales'

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 663974b..58b0d52 100644
--- a/core/src/main/java/org/apache/calcite/schema/SchemaPlus.java
+++ b/core/src/main/java/org/apache/calcite/schema/SchemaPlus.java
@@ -14,9 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq;
+package org.apache.calcite.schema;
 
-import net.hydromatic.optiq.materialize.Lattice;
+import org.apache.calcite.materialize.Lattice;
 
 import com.google.common.collect.ImmutableList;
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/schema/Schemas.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/schema/Schemas.java b/core/src/main/java/org/apache/calcite/schema/Schemas.java
index 00233d7..42df429 100644
--- a/core/src/main/java/org/apache/calcite/schema/Schemas.java
+++ b/core/src/main/java/org/apache/calcite/schema/Schemas.java
@@ -14,55 +14,71 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq;
-
-import net.hydromatic.linq4j.Enumerable;
-import net.hydromatic.linq4j.QueryProvider;
-import net.hydromatic.linq4j.Queryable;
-import net.hydromatic.linq4j.expressions.*;
-
-import net.hydromatic.optiq.config.OptiqConnectionConfig;
-import net.hydromatic.optiq.config.OptiqConnectionConfigImpl;
-import net.hydromatic.optiq.config.OptiqConnectionProperty;
-import net.hydromatic.optiq.impl.java.JavaTypeFactory;
-import net.hydromatic.optiq.jdbc.*;
-import net.hydromatic.optiq.materialize.Lattice;
-
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.reltype.RelDataTypeFactory;
-import org.eigenbase.reltype.RelProtoDataType;
-import org.eigenbase.rex.RexNode;
-import org.eigenbase.sql.type.SqlTypeUtil;
+package org.apache.calcite.schema;
+
+import org.apache.calcite.DataContext;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.config.CalciteConnectionConfig;
+import org.apache.calcite.config.CalciteConnectionConfigImpl;
+import org.apache.calcite.config.CalciteConnectionProperty;
+import org.apache.calcite.jdbc.CalciteConnection;
+import org.apache.calcite.jdbc.CalcitePrepare;
+import org.apache.calcite.jdbc.CalciteRootSchema;
+import org.apache.calcite.jdbc.CalciteSchema;
+import org.apache.calcite.linq4j.Enumerable;
+import org.apache.calcite.linq4j.QueryProvider;
+import org.apache.calcite.linq4j.Queryable;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.linq4j.tree.MethodCallExpression;
+import org.apache.calcite.linq4j.tree.Types;
+import org.apache.calcite.materialize.Lattice;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelProtoDataType;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.type.SqlTypeUtil;
+import org.apache.calcite.util.BuiltInMethod;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Lists;
 
-import java.lang.reflect.*;
+import java.lang.reflect.Type;
 import java.sql.Connection;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.TimeZone;
 
 /**
  * Utility functions for schemas.
  */
 public final class Schemas {
-  private static final com.google.common.base.Function<OptiqSchema.LatticeEntry,
-      OptiqSchema.TableEntry> TO_TABLE_ENTRY =
-      new com.google.common.base.Function<OptiqSchema.LatticeEntry,
-          OptiqSchema.TableEntry>() {
-        public OptiqSchema.TableEntry apply(OptiqSchema.LatticeEntry entry) {
-          final OptiqSchema.TableEntry starTable = entry.getStarTable();
+  private static final com.google.common.base.Function<
+      CalciteSchema.LatticeEntry,
+      CalciteSchema.TableEntry> TO_TABLE_ENTRY =
+      new com.google.common.base.Function<CalciteSchema.LatticeEntry,
+          CalciteSchema.TableEntry>() {
+        public CalciteSchema.TableEntry apply(
+            CalciteSchema.LatticeEntry entry) {
+          final CalciteSchema.TableEntry starTable = entry.getStarTable();
           assert starTable.getTable().getJdbcTableType()
               == Schema.TableType.STAR;
           return entry.getStarTable();
         }
       };
 
-  private static final com.google.common.base.Function<OptiqSchema.LatticeEntry,
+  private static final com.google.common.base.Function<
+      CalciteSchema.LatticeEntry,
       Lattice> TO_LATTICE =
-      new com.google.common.base.Function<OptiqSchema.LatticeEntry, Lattice>() {
-        public Lattice apply(OptiqSchema.LatticeEntry entry) {
+      new com.google.common.base.Function<CalciteSchema.LatticeEntry,
+          Lattice>() {
+        public Lattice apply(CalciteSchema.LatticeEntry entry) {
           return entry.getLattice();
         }
       };
@@ -71,14 +87,14 @@ public final class Schemas {
     throw new AssertionError("no instances!");
   }
 
-  public static OptiqSchema.FunctionEntry resolve(
+  public static CalciteSchema.FunctionEntry resolve(
       RelDataTypeFactory typeFactory,
       String name,
-      Collection<OptiqSchema.FunctionEntry> functionEntries,
+      Collection<CalciteSchema.FunctionEntry> functionEntries,
       List<RelDataType> argumentTypes) {
-    final List<OptiqSchema.FunctionEntry> matches =
-        new ArrayList<OptiqSchema.FunctionEntry>();
-    for (OptiqSchema.FunctionEntry entry : functionEntries) {
+    final List<CalciteSchema.FunctionEntry> matches =
+        new ArrayList<CalciteSchema.FunctionEntry>();
+    for (CalciteSchema.FunctionEntry entry : functionEntries) {
       if (matches(typeFactory, entry.getFunction(), argumentTypes)) {
         matches.add(entry);
       }
@@ -89,8 +105,7 @@ public final class Schemas {
     case 1:
       return matches.get(0);
     default:
-      throw new RuntimeException(
-          "More than one match for " + name
+      throw new RuntimeException("More than one match for " + name
           + " with arguments " + argumentTypes);
     }
   }
@@ -128,7 +143,7 @@ public final class Schemas {
     Expression call =
         Expressions.call(
             schemaExpression,
-            BuiltinMethod.SCHEMA_GET_SUB_SCHEMA.method,
+            BuiltInMethod.SCHEMA_GET_SUB_SCHEMA.method,
             Expressions.constant(name));
     //CHECKSTYLE: IGNORE 2
     //noinspection unchecked
@@ -139,10 +154,10 @@ public final class Schemas {
   }
 
   /** Converts a schema expression to a given type by calling the
-   * {@link net.hydromatic.optiq.SchemaPlus#unwrap(Class)} method. */
+   * {@link SchemaPlus#unwrap(Class)} method. */
   public static Expression unwrap(Expression call, Class type) {
     return Expressions.convert_(
-        Expressions.call(call, BuiltinMethod.SCHEMA_PLUS_UNWRAP.method,
+        Expressions.call(call, BuiltInMethod.SCHEMA_PLUS_UNWRAP.method,
             Expressions.constant(type)),
         type);
   }
@@ -154,23 +169,23 @@ public final class Schemas {
     if (Table.class.isAssignableFrom(clazz)) {
       expression = Expressions.call(
           expression(schema),
-          BuiltinMethod.SCHEMA_GET_TABLE.method,
+          BuiltInMethod.SCHEMA_GET_TABLE.method,
           Expressions.constant(tableName));
       if (ScannableTable.class.isAssignableFrom(clazz)) {
         return Expressions.call(
-            BuiltinMethod.SCHEMAS_ENUMERABLE.method,
+            BuiltInMethod.SCHEMAS_ENUMERABLE.method,
             Expressions.convert_(expression, ScannableTable.class),
             DataContext.ROOT);
       }
       if (FilterableTable.class.isAssignableFrom(clazz)) {
         return Expressions.call(
-            BuiltinMethod.SCHEMAS_ENUMERABLE2.method,
+            BuiltInMethod.SCHEMAS_ENUMERABLE2.method,
             Expressions.convert_(expression, FilterableTable.class),
             DataContext.ROOT);
       }
     } else {
       expression = Expressions.call(
-          BuiltinMethod.SCHEMAS_QUERYABLE.method,
+          BuiltInMethod.SCHEMAS_QUERYABLE.method,
           DataContext.ROOT,
           expression(schema),
           Expressions.constant(elementType),
@@ -180,7 +195,7 @@ public final class Schemas {
   }
 
   public static DataContext createDataContext(Connection connection) {
-    return new DummyDataContext((OptiqConnection) connection);
+    return new DummyDataContext((CalciteConnection) connection);
   }
 
   /** Returns a {@link Queryable}, given a fully-qualified table name. */
@@ -211,14 +226,14 @@ public final class Schemas {
     return table.asQueryable(root.getQueryProvider(), schema, tableName);
   }
 
-  /** Returns an {@link net.hydromatic.linq4j.Enumerable} over the rows of
+  /** Returns an {@link org.apache.calcite.linq4j.Enumerable} over the rows of
    * a given table, representing each row as an object array. */
   public static Enumerable<Object[]> enumerable(final ScannableTable table,
       final DataContext root) {
     return table.scan(root);
   }
 
-  /** Returns an {@link net.hydromatic.linq4j.Enumerable} over the rows of
+  /** Returns an {@link org.apache.calcite.linq4j.Enumerable} over the rows of
    * a given table, not applying any filters, representing each row as an object
    * array. */
   public static Enumerable<Object[]> enumerable(final FilterableTable table,
@@ -226,8 +241,8 @@ public final class Schemas {
     return table.scan(root, ImmutableList.<RexNode>of());
   }
 
-  /** Returns an {@link net.hydromatic.linq4j.Enumerable} over object arrays,
-   * given a fully-qualified table name which leads to a
+  /** Returns an {@link org.apache.calcite.linq4j.Enumerable} over object
+   * arrays, given a fully-qualified table name which leads to a
    * {@link ScannableTable}. */
   public static Table table(DataContext root, String... names) {
     SchemaPlus schema = root.getRootSchema();
@@ -243,93 +258,94 @@ public final class Schemas {
   }
 
   /** Parses and validates a SQL query. For use within Calcite only. */
-  public static OptiqPrepare.ParseResult parse(
-      final OptiqConnection connection, final OptiqSchema schema,
+  public static CalcitePrepare.ParseResult parse(
+      final CalciteConnection connection, final CalciteSchema schema,
       final List<String> schemaPath, final String sql) {
-    final OptiqPrepare prepare = OptiqPrepare.DEFAULT_FACTORY.apply();
-    final OptiqPrepare.Context context =
+    final CalcitePrepare prepare = CalcitePrepare.DEFAULT_FACTORY.apply();
+    final CalcitePrepare.Context context =
         makeContext(connection, schema, schemaPath,
-            ImmutableMap.<OptiqConnectionProperty, String>of());
-    OptiqPrepare.Dummy.push(context);
+            ImmutableMap.<CalciteConnectionProperty, String>of());
+    CalcitePrepare.Dummy.push(context);
     try {
       return prepare.parse(context, sql);
     } finally {
-      OptiqPrepare.Dummy.pop(context);
+      CalcitePrepare.Dummy.pop(context);
     }
   }
 
   /** Parses and validates a SQL query and converts to relational algebra. For
    * use within Calcite only. */
-  public static OptiqPrepare.ConvertResult convert(
-      final OptiqConnection connection, final OptiqSchema schema,
+  public static CalcitePrepare.ConvertResult convert(
+      final CalciteConnection connection, final CalciteSchema schema,
       final List<String> schemaPath, final String sql) {
-    final OptiqPrepare prepare = OptiqPrepare.DEFAULT_FACTORY.apply();
-    final OptiqPrepare.Context context =
+    final CalcitePrepare prepare = CalcitePrepare.DEFAULT_FACTORY.apply();
+    final CalcitePrepare.Context context =
         makeContext(connection, schema, schemaPath,
-            ImmutableMap.<OptiqConnectionProperty, String>of());
-    OptiqPrepare.Dummy.push(context);
+            ImmutableMap.<CalciteConnectionProperty, String>of());
+    CalcitePrepare.Dummy.push(context);
     try {
       return prepare.convert(context, sql);
     } finally {
-      OptiqPrepare.Dummy.pop(context);
+      CalcitePrepare.Dummy.pop(context);
     }
   }
 
   /** Prepares a SQL query for execution. For use within Calcite only. */
-  public static OptiqPrepare.PrepareResult<Object> prepare(
-      final OptiqConnection connection, final OptiqSchema schema,
+  public static CalcitePrepare.PrepareResult<Object> prepare(
+      final CalciteConnection connection, final CalciteSchema schema,
       final List<String> schemaPath, final String sql,
-      final ImmutableMap<OptiqConnectionProperty, String> map) {
-    final OptiqPrepare prepare = OptiqPrepare.DEFAULT_FACTORY.apply();
-    final OptiqPrepare.Context context =
+      final ImmutableMap<CalciteConnectionProperty, String> map) {
+    final CalcitePrepare prepare = CalcitePrepare.DEFAULT_FACTORY.apply();
+    final CalcitePrepare.Context context =
         makeContext(connection, schema, schemaPath, map);
-    OptiqPrepare.Dummy.push(context);
+    CalcitePrepare.Dummy.push(context);
     try {
       return prepare.prepareSql(context, sql, null, Object[].class, -1);
     } finally {
-      OptiqPrepare.Dummy.pop(context);
+      CalcitePrepare.Dummy.pop(context);
     }
   }
 
-  public static OptiqPrepare.Context makeContext(
-      final OptiqConnection connection, final OptiqSchema schema,
+  public static CalcitePrepare.Context makeContext(
+      final CalciteConnection connection, final CalciteSchema schema,
       final List<String> schemaPath,
-      final ImmutableMap<OptiqConnectionProperty, String> propValues) {
+      final ImmutableMap<CalciteConnectionProperty, String> propValues) {
     if (connection == null) {
-      final OptiqPrepare.Context context0 = OptiqPrepare.Dummy.peek();
-      final OptiqConnectionConfig config =
+      final CalcitePrepare.Context context0 = CalcitePrepare.Dummy.peek();
+      final CalciteConnectionConfig config =
           mutate(context0.config(), propValues);
       return makeContext(config, context0.getTypeFactory(),
           context0.getDataContext(), schema, schemaPath);
     } else {
-      final OptiqConnectionConfig config =
+      final CalciteConnectionConfig config =
           mutate(connection.config(), propValues);
       return makeContext(config, connection.getTypeFactory(),
           createDataContext(connection), schema, schemaPath);
     }
   }
 
-  private static OptiqConnectionConfig mutate(OptiqConnectionConfig config,
-      ImmutableMap<OptiqConnectionProperty, String> propValues) {
-    for (Map.Entry<OptiqConnectionProperty, String> e : propValues.entrySet()) {
+  private static CalciteConnectionConfig mutate(CalciteConnectionConfig config,
+      ImmutableMap<CalciteConnectionProperty, String> propValues) {
+    for (Map.Entry<CalciteConnectionProperty, String> e
+        : propValues.entrySet()) {
       config =
-          ((OptiqConnectionConfigImpl) config).set(e.getKey(), e.getValue());
+          ((CalciteConnectionConfigImpl) config).set(e.getKey(), e.getValue());
     }
     return config;
   }
 
-  private static OptiqPrepare.Context makeContext(
-      final OptiqConnectionConfig connectionConfig,
+  private static CalcitePrepare.Context makeContext(
+      final CalciteConnectionConfig connectionConfig,
       final JavaTypeFactory typeFactory,
       final DataContext dataContext,
-      final OptiqSchema schema,
+      final CalciteSchema schema,
       final List<String> schemaPath) {
-    return new OptiqPrepare.Context() {
+    return new CalcitePrepare.Context() {
       public JavaTypeFactory getTypeFactory() {
         return typeFactory;
       }
 
-      public OptiqRootSchema getRootSchema() {
+      public CalciteRootSchema getRootSchema() {
         return schema.root();
       }
 
@@ -342,7 +358,7 @@ public final class Schemas {
         return schemaPath;
       }
 
-      public OptiqConnectionConfig config() {
+      public CalciteConnectionConfig config() {
         return connectionConfig;
       }
 
@@ -350,9 +366,9 @@ public final class Schemas {
         return dataContext;
       }
 
-      public OptiqPrepare.SparkHandler spark() {
+      public CalcitePrepare.SparkHandler spark() {
         final boolean enable = config().spark();
-        return OptiqPrepare.Dummy.getSparkHandler(enable);
+        return CalcitePrepare.Dummy.getSparkHandler(enable);
       }
     };
   }
@@ -382,38 +398,40 @@ public final class Schemas {
   /** Returns the star tables defined in a schema.
    *
    * @param schema Schema */
-  public static List<OptiqSchema.TableEntry> getStarTables(OptiqSchema schema) {
-    final List<OptiqSchema.LatticeEntry> list = getLatticeEntries(schema);
+  public static List<CalciteSchema.TableEntry>
+  getStarTables(CalciteSchema schema) {
+    final List<CalciteSchema.LatticeEntry> list = getLatticeEntries(schema);
     return Lists.transform(list, TO_TABLE_ENTRY);
   }
 
   /** Returns the lattices defined in a schema.
    *
    * @param schema Schema */
-  public static List<Lattice> getLattices(OptiqSchema schema) {
-    final List<OptiqSchema.LatticeEntry> list = getLatticeEntries(schema);
+  public static List<Lattice> getLattices(CalciteSchema schema) {
+    final List<CalciteSchema.LatticeEntry> list = getLatticeEntries(schema);
     return Lists.transform(list, TO_LATTICE);
   }
 
   /** Returns the lattices defined in a schema.
    *
    * @param schema Schema */
-  public static List<OptiqSchema.LatticeEntry> getLatticeEntries(
-      OptiqSchema schema) {
-    final List<OptiqSchema.LatticeEntry> list = Lists.newArrayList();
+  public static List<CalciteSchema.LatticeEntry> getLatticeEntries(
+      CalciteSchema schema) {
+    final List<CalciteSchema.LatticeEntry> list = Lists.newArrayList();
     gatherLattices(schema, list);
     return list;
   }
 
-  private static void gatherLattices(OptiqSchema schema,
-      List<OptiqSchema.LatticeEntry> list) {
+  private static void gatherLattices(CalciteSchema schema,
+      List<CalciteSchema.LatticeEntry> list) {
     list.addAll(schema.getLatticeMap().values());
-    for (OptiqSchema subSchema : schema.getSubSchemaMap().values()) {
+    for (CalciteSchema subSchema : schema.getSubSchemaMap().values()) {
       gatherLattices(subSchema, list);
     }
   }
 
-  public static OptiqSchema subSchema(OptiqSchema schema, List<String> names) {
+  public static CalciteSchema subSchema(CalciteSchema schema,
+        List<String> names) {
     for (String string : names) {
       schema = schema.getSubSchema(string, false);
     }
@@ -421,7 +439,7 @@ public final class Schemas {
   }
 
   /** Generates a table name that is unique within the given schema. */
-  public static String uniqueTableName(OptiqSchema schema, String base) {
+  public static String uniqueTableName(CalciteSchema schema, String base) {
     String t = Preconditions.checkNotNull(base);
     for (int x = 0; schema.getTable(t, true) != null; x++) {
       t = base + x;
@@ -431,10 +449,10 @@ public final class Schemas {
 
   /** Dummy data context that has no variables. */
   private static class DummyDataContext implements DataContext {
-    private final OptiqConnection connection;
+    private final CalciteConnection connection;
     private final ImmutableMap<String, Object> map;
 
-    public DummyDataContext(OptiqConnection connection) {
+    public DummyDataContext(CalciteConnection connection) {
       this.connection = connection;
       this.map =
           ImmutableMap.<String, Object>of("timeZone", TimeZone.getDefault());

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/schema/SemiMutableSchema.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/schema/SemiMutableSchema.java b/core/src/main/java/org/apache/calcite/schema/SemiMutableSchema.java
index c39e197..0865cff 100644
--- a/core/src/main/java/org/apache/calcite/schema/SemiMutableSchema.java
+++ b/core/src/main/java/org/apache/calcite/schema/SemiMutableSchema.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq;
+package org.apache.calcite.schema;
 
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/schema/Statistic.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/schema/Statistic.java b/core/src/main/java/org/apache/calcite/schema/Statistic.java
index 8286ae3..fa907da 100644
--- a/core/src/main/java/org/apache/calcite/schema/Statistic.java
+++ b/core/src/main/java/org/apache/calcite/schema/Statistic.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq;
+package org.apache.calcite.schema;
 
 import java.util.BitSet;
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/schema/Statistics.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/schema/Statistics.java b/core/src/main/java/org/apache/calcite/schema/Statistics.java
index ab5f93f..9712e7c 100644
--- a/core/src/main/java/org/apache/calcite/schema/Statistics.java
+++ b/core/src/main/java/org/apache/calcite/schema/Statistics.java
@@ -14,11 +14,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq;
+package org.apache.calcite.schema;
 
-import net.hydromatic.optiq.util.BitSets;
+import org.apache.calcite.util.BitSets;
 
-import java.util.*;
+import java.util.BitSet;
+import java.util.List;
 
 /**
  * Utility functions regarding {@link Statistic}.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/schema/Table.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/schema/Table.java b/core/src/main/java/org/apache/calcite/schema/Table.java
index 4e8f76c..b25745e 100644
--- a/core/src/main/java/org/apache/calcite/schema/Table.java
+++ b/core/src/main/java/org/apache/calcite/schema/Table.java
@@ -14,18 +14,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq;
+package org.apache.calcite.schema;
 
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.reltype.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
 
 /**
  * Table.
  *
- * <p>The typical way for a table to be created is when Calcite interrogates
- * a user-defined schema in order to validate names appearing in a SQL query.
- * Calcite finds the schema by calling {@link Schema#getSubSchema(String)} on the
- * connection's root schema, then gets a table by calling
+ * <p>The typical way for a table to be created is when Calcite interrogates a
+ * user-defined schema in order to validate names appearing in a SQL query.
+ * Calcite finds the schema by calling {@link Schema#getSubSchema(String)} on
+ * the connection's root schema, then gets a table by calling
  * {@link Schema#getTable(String)}.</p>
  *
  * <p>Note that a table does not know its name. It is in fact possible for

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/schema/TableFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/schema/TableFactory.java b/core/src/main/java/org/apache/calcite/schema/TableFactory.java
index ccb0c33..ec0321a 100644
--- a/core/src/main/java/org/apache/calcite/schema/TableFactory.java
+++ b/core/src/main/java/org/apache/calcite/schema/TableFactory.java
@@ -14,9 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq;
+package org.apache.calcite.schema;
 
-import org.eigenbase.reltype.RelDataType;
+import org.apache.calcite.rel.type.RelDataType;
 
 import java.util.Map;
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/schema/TableFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/schema/TableFunction.java b/core/src/main/java/org/apache/calcite/schema/TableFunction.java
index fe28921..96ef00b 100644
--- a/core/src/main/java/org/apache/calcite/schema/TableFunction.java
+++ b/core/src/main/java/org/apache/calcite/schema/TableFunction.java
@@ -14,18 +14,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq;
+package org.apache.calcite.schema;
 
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.reltype.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
 
 import java.lang.reflect.Type;
 import java.util.List;
 
 /**
  * Function that returns a table during execution time.
- * <p>
- * In contrast with {@code TableMacro}, the result of the table is not known
+ *
+ * <p>In contrast with {@code TableMacro}, the result of the table is not known
  * until execution.
  */
 public interface TableFunction extends Function {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/schema/TableMacro.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/schema/TableMacro.java b/core/src/main/java/org/apache/calcite/schema/TableMacro.java
index 2a88886..8288909 100644
--- a/core/src/main/java/org/apache/calcite/schema/TableMacro.java
+++ b/core/src/main/java/org/apache/calcite/schema/TableMacro.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq;
+package org.apache.calcite.schema;
 
 import java.util.List;
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/schema/TranslatableTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/schema/TranslatableTable.java b/core/src/main/java/org/apache/calcite/schema/TranslatableTable.java
index f7d126b..0a254ba 100644
--- a/core/src/main/java/org/apache/calcite/schema/TranslatableTable.java
+++ b/core/src/main/java/org/apache/calcite/schema/TranslatableTable.java
@@ -14,14 +14,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq;
+package org.apache.calcite.schema;
 
-import org.eigenbase.rel.RelNode;
-import org.eigenbase.relopt.RelOptTable;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.rel.RelNode;
 
 /**
  * Extension to {@link Table} that specifies how it is to be translated to
- * a {@link org.eigenbase.rel.RelNode planner node}.
+ * a {@link org.apache.calcite.rel.RelNode planner node}.
  *
  * <p>It is optional for a Table to implement this interface. A Table that does
  * not implement this interface, a Table will be converted to an

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/schema/impl/AbstractSchema.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/schema/impl/AbstractSchema.java b/core/src/main/java/org/apache/calcite/schema/impl/AbstractSchema.java
index 013a01e..ca3389b 100644
--- a/core/src/main/java/org/apache/calcite/schema/impl/AbstractSchema.java
+++ b/core/src/main/java/org/apache/calcite/schema/impl/AbstractSchema.java
@@ -14,16 +14,22 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl;
-
-import net.hydromatic.linq4j.expressions.Expression;
-
-import net.hydromatic.optiq.*;
-import net.hydromatic.optiq.Table;
-
-import com.google.common.collect.*;
-
-import java.util.*;
+package org.apache.calcite.schema.impl;
+
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.schema.Function;
+import org.apache.calcite.schema.Schema;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.schema.Schemas;
+import org.apache.calcite.schema.Table;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableMultimap;
+import com.google.common.collect.Multimap;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
 
 /**
  * Abstract implementation of {@link Schema}.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/schema/impl/AbstractTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/schema/impl/AbstractTable.java b/core/src/main/java/org/apache/calcite/schema/impl/AbstractTable.java
index 82900b6..3a96dc9 100644
--- a/core/src/main/java/org/apache/calcite/schema/impl/AbstractTable.java
+++ b/core/src/main/java/org/apache/calcite/schema/impl/AbstractTable.java
@@ -14,9 +14,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl;
+package org.apache.calcite.schema.impl;
 
-import net.hydromatic.optiq.*;
+import org.apache.calcite.schema.Schema;
+import org.apache.calcite.schema.Statistic;
+import org.apache.calcite.schema.Statistics;
+import org.apache.calcite.schema.Table;
 
 /**
  * Abstract base class for implementing {@link Table}.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/schema/impl/AbstractTableQueryable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/schema/impl/AbstractTableQueryable.java b/core/src/main/java/org/apache/calcite/schema/impl/AbstractTableQueryable.java
index 78c04d2..1e8774c 100644
--- a/core/src/main/java/org/apache/calcite/schema/impl/AbstractTableQueryable.java
+++ b/core/src/main/java/org/apache/calcite/schema/impl/AbstractTableQueryable.java
@@ -14,23 +14,25 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl;
+package org.apache.calcite.schema.impl;
 
-import net.hydromatic.linq4j.*;
-import net.hydromatic.linq4j.expressions.Expression;
-
-import net.hydromatic.optiq.QueryableTable;
-import net.hydromatic.optiq.SchemaPlus;
+import org.apache.calcite.linq4j.AbstractQueryable;
+import org.apache.calcite.linq4j.Linq4j;
+import org.apache.calcite.linq4j.QueryProvider;
+import org.apache.calcite.linq4j.Queryable;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.schema.QueryableTable;
+import org.apache.calcite.schema.SchemaPlus;
 
 import java.lang.reflect.Type;
 import java.util.Iterator;
 
 /**
- * Abstract implementation of {@link net.hydromatic.linq4j.Queryable} for
+ * Abstract implementation of {@link org.apache.calcite.linq4j.Queryable} for
  * {@link QueryableTable}.
  *
  * <p>Not to be confused with
- * {@link net.hydromatic.optiq.impl.java.AbstractQueryableTable}.</p>
+ * {@link org.apache.calcite.adapter.java.AbstractQueryableTable}.</p>
  *
  * @param <T> element type
  */

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/schema/impl/AggregateFunctionImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/schema/impl/AggregateFunctionImpl.java b/core/src/main/java/org/apache/calcite/schema/impl/AggregateFunctionImpl.java
index 4e3e4c5..a2a4140 100644
--- a/core/src/main/java/org/apache/calcite/schema/impl/AggregateFunctionImpl.java
+++ b/core/src/main/java/org/apache/calcite/schema/impl/AggregateFunctionImpl.java
@@ -14,14 +14,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl;
+package org.apache.calcite.schema.impl;
 
-import net.hydromatic.optiq.*;
-import net.hydromatic.optiq.rules.java.*;
-
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.reltype.RelDataTypeFactory;
-import org.eigenbase.util.Util;
+import org.apache.calcite.adapter.enumerable.AggImplementor;
+import org.apache.calcite.adapter.enumerable.RexImpTable;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.schema.AggregateFunction;
+import org.apache.calcite.schema.FunctionParameter;
+import org.apache.calcite.schema.ImplementableAggFunction;
+import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
 
@@ -29,7 +31,7 @@ import java.lang.reflect.Method;
 import java.lang.reflect.Modifier;
 import java.util.List;
 
-import static org.eigenbase.util.Static.*;
+import static org.apache.calcite.util.Static.RESOURCE;
 
 /**
  * Implementation of {@link AggregateFunction} via user-defined class.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/schema/impl/DelegatingSchema.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/schema/impl/DelegatingSchema.java b/core/src/main/java/org/apache/calcite/schema/impl/DelegatingSchema.java
index e3d2cd5..0612a59 100644
--- a/core/src/main/java/org/apache/calcite/schema/impl/DelegatingSchema.java
+++ b/core/src/main/java/org/apache/calcite/schema/impl/DelegatingSchema.java
@@ -14,18 +14,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl;
+package org.apache.calcite.schema.impl;
 
-import net.hydromatic.linq4j.expressions.Expression;
-
-import net.hydromatic.optiq.*;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.schema.Function;
+import org.apache.calcite.schema.Schema;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.schema.Table;
 
 import java.util.Collection;
 import java.util.Set;
 
 /**
- * Implementation of {@link net.hydromatic.optiq.Schema} that delegates to an
- * underlying schema.
+ * Implementation of {@link org.apache.calcite.schema.Schema} that delegates to
+ * an underlying schema.
  */
 public class DelegatingSchema implements Schema {
   protected final Schema schema;
@@ -39,8 +41,7 @@ public class DelegatingSchema implements Schema {
     this.schema = schema;
   }
 
-  @Override
-  public String toString() {
+  @Override public String toString() {
     return "DelegatingSchema(delegate=" + schema + ")";
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/schema/impl/MaterializedViewTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/schema/impl/MaterializedViewTable.java b/core/src/main/java/org/apache/calcite/schema/impl/MaterializedViewTable.java
index aac77b5..5810f76 100644
--- a/core/src/main/java/org/apache/calcite/schema/impl/MaterializedViewTable.java
+++ b/core/src/main/java/org/apache/calcite/schema/impl/MaterializedViewTable.java
@@ -14,20 +14,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl;
+package org.apache.calcite.schema.impl;
 
-import net.hydromatic.optiq.*;
-import net.hydromatic.optiq.impl.java.JavaTypeFactory;
-import net.hydromatic.optiq.jdbc.OptiqConnection;
-import net.hydromatic.optiq.jdbc.OptiqPrepare;
-import net.hydromatic.optiq.jdbc.OptiqSchema;
-import net.hydromatic.optiq.materialize.MaterializationKey;
-import net.hydromatic.optiq.materialize.MaterializationService;
-
-import org.eigenbase.rel.RelNode;
-import org.eigenbase.relopt.RelOptTable;
-import org.eigenbase.reltype.RelDataTypeImpl;
-import org.eigenbase.reltype.RelProtoDataType;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.jdbc.CalciteConnection;
+import org.apache.calcite.jdbc.CalcitePrepare;
+import org.apache.calcite.jdbc.CalciteSchema;
+import org.apache.calcite.materialize.MaterializationKey;
+import org.apache.calcite.materialize.MaterializationService;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.type.RelDataTypeImpl;
+import org.apache.calcite.rel.type.RelProtoDataType;
+import org.apache.calcite.schema.Schemas;
+import org.apache.calcite.schema.Table;
+import org.apache.calcite.schema.TranslatableTable;
 
 import com.google.common.base.Preconditions;
 
@@ -42,7 +43,7 @@ import java.util.List;
  * <p>It can exist in two states: materialized and not materialized. Over time,
  * a given materialized view may switch states. How it is expanded depends upon
  * its current state. State is managed by
- * {@link net.hydromatic.optiq.materialize.MaterializationService}.</p>
+ * {@link org.apache.calcite.materialize.MaterializationService}.</p>
  */
 public class MaterializedViewTable extends ViewTable {
 
@@ -52,12 +53,12 @@ public class MaterializedViewTable extends ViewTable {
    * Internal connection, used to execute queries to materialize views.
    * To be used only by Calcite internals. And sparingly.
    */
-  public static final OptiqConnection MATERIALIZATION_CONNECTION;
+  public static final CalciteConnection MATERIALIZATION_CONNECTION;
 
   static {
     try {
       MATERIALIZATION_CONNECTION = DriverManager.getConnection("jdbc:calcite:")
-          .unwrap(OptiqConnection.class);
+          .unwrap(CalciteConnection.class);
     } catch (SQLException e) {
       throw new RuntimeException(e);
     }
@@ -73,7 +74,7 @@ public class MaterializedViewTable extends ViewTable {
   }
 
   /** Table macro that returns a materialized view. */
-  public static MaterializedViewTableMacro create(final OptiqSchema schema,
+  public static MaterializedViewTableMacro create(final CalciteSchema schema,
       final String viewSql,
       final List<String> viewSchemaPath,
       final String tableName) {
@@ -81,10 +82,9 @@ public class MaterializedViewTable extends ViewTable {
         tableName);
   }
 
-  @Override
-  public RelNode toRel(RelOptTable.ToRelContext context,
+  @Override public RelNode toRel(RelOptTable.ToRelContext context,
       RelOptTable relOptTable) {
-    final OptiqSchema.TableEntry tableEntry =
+    final CalciteSchema.TableEntry tableEntry =
         MaterializationService.instance().checkValid(key);
     if (tableEntry != null) {
       Table materializeTable = tableEntry.getTable();
@@ -101,7 +101,7 @@ public class MaterializedViewTable extends ViewTable {
       extends ViewTableMacro {
     private final MaterializationKey key;
 
-    private MaterializedViewTableMacro(OptiqSchema schema, String viewSql,
+    private MaterializedViewTableMacro(CalciteSchema schema, String viewSql,
         List<String> viewSchemaPath, String suggestedTableName) {
       super(schema, viewSql, viewSchemaPath);
       this.key = Preconditions.checkNotNull(
@@ -109,10 +109,9 @@ public class MaterializedViewTable extends ViewTable {
               schema, null, viewSql, schemaPath, suggestedTableName, true));
     }
 
-    @Override
-    public TranslatableTable apply(List<Object> arguments) {
+    @Override public TranslatableTable apply(List<Object> arguments) {
       assert arguments.isEmpty();
-      OptiqPrepare.ParseResult parsed =
+      CalcitePrepare.ParseResult parsed =
           Schemas.parse(MATERIALIZATION_CONNECTION, schema, schemaPath,
               viewSql);
       final List<String> schemaPath1 =

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/schema/impl/ReflectiveFunctionBase.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/schema/impl/ReflectiveFunctionBase.java b/core/src/main/java/org/apache/calcite/schema/impl/ReflectiveFunctionBase.java
index 12e8f90..681b69b 100644
--- a/core/src/main/java/org/apache/calcite/schema/impl/ReflectiveFunctionBase.java
+++ b/core/src/main/java/org/apache/calcite/schema/impl/ReflectiveFunctionBase.java
@@ -14,13 +14,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl;
+package org.apache.calcite.schema.impl;
 
-import net.hydromatic.optiq.Function;
-import net.hydromatic.optiq.FunctionParameter;
-
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.reltype.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.schema.Function;
+import org.apache.calcite.schema.FunctionParameter;
 
 import com.google.common.collect.ImmutableList;
 
@@ -122,4 +121,3 @@ public abstract class ReflectiveFunctionBase implements Function {
 }
 
 // End ReflectiveFunctionBase.java
-

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/schema/impl/ScalarFunctionImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/schema/impl/ScalarFunctionImpl.java b/core/src/main/java/org/apache/calcite/schema/impl/ScalarFunctionImpl.java
index a451f7a..09dfb67 100644
--- a/core/src/main/java/org/apache/calcite/schema/impl/ScalarFunctionImpl.java
+++ b/core/src/main/java/org/apache/calcite/schema/impl/ScalarFunctionImpl.java
@@ -14,22 +14,26 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl;
+package org.apache.calcite.schema.impl;
 
-import net.hydromatic.optiq.*;
-import net.hydromatic.optiq.rules.java.*;
-
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.reltype.RelDataTypeFactory;
+import org.apache.calcite.adapter.enumerable.CallImplementor;
+import org.apache.calcite.adapter.enumerable.NullPolicy;
+import org.apache.calcite.adapter.enumerable.ReflectiveCallNotNullImplementor;
+import org.apache.calcite.adapter.enumerable.RexImpTable;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.schema.ImplementableFunction;
+import org.apache.calcite.schema.ScalarFunction;
 
 import com.google.common.collect.ImmutableMultimap;
 
-import java.lang.reflect.*;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
 
-import static org.eigenbase.util.Static.*;
+import static org.apache.calcite.util.Static.RESOURCE;
 
 /**
-* Implementation of {@link net.hydromatic.optiq.ScalarFunction}.
+* Implementation of {@link org.apache.calcite.schema.ScalarFunction}.
 */
 public class ScalarFunctionImpl extends ReflectiveFunctionBase implements
     ScalarFunction, ImplementableFunction {
@@ -42,8 +46,8 @@ public class ScalarFunctionImpl extends ReflectiveFunctionBase implements
   }
 
   /**
-   * Creates {@link net.hydromatic.optiq.ScalarFunction} for each method in a
-   * given class.
+   * Creates {@link org.apache.calcite.schema.ScalarFunction} for each method in
+   * a given class.
    */
   public static ImmutableMultimap<String, ScalarFunction> createAll(
       Class<?> clazz) {
@@ -54,7 +58,7 @@ public class ScalarFunctionImpl extends ReflectiveFunctionBase implements
         continue;
       }
       if (!Modifier.isStatic(method.getModifiers())
-          && !ScalarFunctionImpl.classHasPublicZeroArgsConstructor(clazz)) {
+          && !classHasPublicZeroArgsConstructor(clazz)) {
         continue;
       }
       final ScalarFunction function = create(method);
@@ -64,7 +68,7 @@ public class ScalarFunctionImpl extends ReflectiveFunctionBase implements
   }
 
   /**
-   * Creates {@link net.hydromatic.optiq.ScalarFunction} from given class.
+   * Creates {@link org.apache.calcite.schema.ScalarFunction} from given class.
    *
    * <p>If a method of the given name is not found or it does not suit,
    * returns {@code null}.
@@ -82,7 +86,7 @@ public class ScalarFunctionImpl extends ReflectiveFunctionBase implements
   }
 
   /**
-   * Creates {@link net.hydromatic.optiq.ScalarFunction} from given method.
+   * Creates {@link org.apache.calcite.schema.ScalarFunction} from given method.
    * When {@code eval} method does not suit, {@code null} is returned.
    *
    * @param method method that is used to implement the function

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/schema/impl/StarTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/schema/impl/StarTable.java b/core/src/main/java/org/apache/calcite/schema/impl/StarTable.java
index a2f31d6..25e7480 100644
--- a/core/src/main/java/org/apache/calcite/schema/impl/StarTable.java
+++ b/core/src/main/java/org/apache/calcite/schema/impl/StarTable.java
@@ -14,19 +14,24 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl;
-
-import net.hydromatic.optiq.Schema;
-import net.hydromatic.optiq.Table;
-import net.hydromatic.optiq.TranslatableTable;
-import net.hydromatic.optiq.materialize.Lattice;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.reltype.RelDataTypeFactory;
-import org.eigenbase.util.ImmutableIntList;
-import org.eigenbase.util.Pair;
+package org.apache.calcite.schema.impl;
+
+import org.apache.calcite.materialize.Lattice;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.schema.Schema;
+import org.apache.calcite.schema.Table;
+import org.apache.calcite.schema.TranslatableTable;
+import org.apache.calcite.util.ImmutableIntList;
+import org.apache.calcite.util.Pair;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
@@ -119,7 +124,7 @@ public class StarTable extends AbstractTable implements TranslatableTable {
    *
    * <p>It has infinite cost.
    */
-  public static class StarTableScan extends TableAccessRelBase {
+  public static class StarTableScan extends TableScan {
     public StarTableScan(RelOptCluster cluster, RelOptTable relOptTable) {
       super(cluster, cluster.traitSetOf(Convention.NONE), relOptTable);
     }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/schema/impl/TableFunctionImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/schema/impl/TableFunctionImpl.java b/core/src/main/java/org/apache/calcite/schema/impl/TableFunctionImpl.java
index f25d9d2..e8b523b 100644
--- a/core/src/main/java/org/apache/calcite/schema/impl/TableFunctionImpl.java
+++ b/core/src/main/java/org/apache/calcite/schema/impl/TableFunctionImpl.java
@@ -14,17 +14,24 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl;
+package org.apache.calcite.schema.impl;
 
-import net.hydromatic.linq4j.expressions.Expression;
-import net.hydromatic.linq4j.expressions.Expressions;
-
-import net.hydromatic.optiq.*;
-import net.hydromatic.optiq.rules.java.*;
-
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.reltype.RelDataTypeFactory;
-import org.eigenbase.rex.RexCall;
+import org.apache.calcite.DataContext;
+import org.apache.calcite.adapter.enumerable.CallImplementor;
+import org.apache.calcite.adapter.enumerable.NullPolicy;
+import org.apache.calcite.adapter.enumerable.ReflectiveCallNotNullImplementor;
+import org.apache.calcite.adapter.enumerable.RexImpTable;
+import org.apache.calcite.adapter.enumerable.RexToLixTranslator;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.schema.ImplementableFunction;
+import org.apache.calcite.schema.QueryableTable;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.schema.TableFunction;
+import org.apache.calcite.util.BuiltInMethod;
 
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
@@ -33,10 +40,10 @@ import java.lang.reflect.Type;
 import java.util.Arrays;
 import java.util.List;
 
-import static org.eigenbase.util.Static.RESOURCE;
+import static org.apache.calcite.util.Static.RESOURCE;
 
 /**
- * Implementation of {@link net.hydromatic.optiq.TableFunction} based on a
+ * Implementation of {@link org.apache.calcite.schema.TableFunction} based on a
  * method.
 */
 public class TableFunctionImpl extends ReflectiveFunctionBase implements
@@ -97,14 +104,14 @@ public class TableFunctionImpl extends ReflectiveFunctionBase implements
             translatedOperands);
         Expression queryable = Expressions.call(
           Expressions.convert_(expr, QueryableTable.class),
-          BuiltinMethod.QUERYABLE_TABLE_AS_QUERYABLE.method,
+          BuiltInMethod.QUERYABLE_TABLE_AS_QUERYABLE.method,
           Expressions.call(DataContext.ROOT,
-            BuiltinMethod.DATA_CONTEXT_GET_QUERY_PROVIDER.method),
+            BuiltInMethod.DATA_CONTEXT_GET_QUERY_PROVIDER.method),
           Expressions.constant(null, SchemaPlus.class),
           Expressions.constant(call.getOperator().getName(),
             String.class));
         expr = Expressions.call(queryable,
-            BuiltinMethod.QUERYABLE_AS_ENUMERABLE.method);
+            BuiltInMethod.QUERYABLE_AS_ENUMERABLE.method);
         return expr;
       }
     }, NullPolicy.ANY, false);

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/schema/impl/TableMacroImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/schema/impl/TableMacroImpl.java b/core/src/main/java/org/apache/calcite/schema/impl/TableMacroImpl.java
index 4728232..609a16f 100644
--- a/core/src/main/java/org/apache/calcite/schema/impl/TableMacroImpl.java
+++ b/core/src/main/java/org/apache/calcite/schema/impl/TableMacroImpl.java
@@ -14,9 +14,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl;
+package org.apache.calcite.schema.impl;
 
-import net.hydromatic.optiq.*;
+import org.apache.calcite.schema.TableMacro;
+import org.apache.calcite.schema.TranslatableTable;
 
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
@@ -24,10 +25,11 @@ import java.lang.reflect.Modifier;
 import java.util.Arrays;
 import java.util.List;
 
-import static org.eigenbase.util.Static.*;
+import static org.apache.calcite.util.Static.RESOURCE;
 
 /**
- * Implementation of {@link net.hydromatic.optiq.TableMacro} based on a method.
+ * Implementation of {@link org.apache.calcite.schema.TableMacro} based on a
+ * method.
 */
 public class TableMacroImpl extends ReflectiveFunctionBase
     implements TableMacro {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 8b710b7..c978a3a 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
@@ -14,21 +14,28 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.impl;
-
-import net.hydromatic.linq4j.QueryProvider;
-import net.hydromatic.linq4j.Queryable;
-
-import net.hydromatic.optiq.*;
-import net.hydromatic.optiq.impl.java.AbstractQueryableTable;
-import net.hydromatic.optiq.impl.java.JavaTypeFactory;
-import net.hydromatic.optiq.jdbc.*;
-
-import org.eigenbase.rel.RelNode;
-import org.eigenbase.relopt.RelOptTable;
-import org.eigenbase.relopt.RelOptUtil;
-import org.eigenbase.reltype.*;
-import org.eigenbase.util.Util;
+package org.apache.calcite.schema.impl;
+
+import org.apache.calcite.adapter.java.AbstractQueryableTable;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.jdbc.CalcitePrepare;
+import org.apache.calcite.jdbc.CalciteSchema;
+import org.apache.calcite.linq4j.QueryProvider;
+import org.apache.calcite.linq4j.Queryable;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeImpl;
+import org.apache.calcite.rel.type.RelProtoDataType;
+import org.apache.calcite.schema.FunctionParameter;
+import org.apache.calcite.schema.Schema;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.schema.Schemas;
+import org.apache.calcite.schema.TableMacro;
+import org.apache.calcite.schema.TranslatableTable;
+import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
 
@@ -36,8 +43,6 @@ import java.lang.reflect.Type;
 import java.util.Collections;
 import java.util.List;
 
-import static net.hydromatic.optiq.impl.MaterializedViewTable.MATERIALIZATION_CONNECTION;
-
 /**
  * Table whose contents are defined using an SQL statement.
  *
@@ -61,7 +66,7 @@ public class ViewTable
   /** Table macro that returns a view. */
   public static ViewTableMacro viewMacro(SchemaPlus schema,
       final String viewSql, final List<String> schemaPath) {
-    return new ViewTableMacro(OptiqSchema.from(schema), viewSql, schemaPath);
+    return new ViewTableMacro(CalciteSchema.from(schema), viewSql, schemaPath);
   }
 
   @Override public Schema.TableType getJdbcTableType() {
@@ -105,12 +110,12 @@ public class ViewTable
    * tree of the view's SQL query. */
   static class ViewTableMacro implements TableMacro {
     protected final String viewSql;
-    protected final OptiqSchema schema;
+    protected final CalciteSchema schema;
     /** Typically null. If specified, overrides the path of the schema as the
      * context for validating {@code viewSql}. */
     protected final List<String> schemaPath;
 
-    ViewTableMacro(OptiqSchema schema, String viewSql,
+    ViewTableMacro(CalciteSchema schema, String viewSql,
         List<String> schemaPath) {
       this.viewSql = viewSql;
       this.schema = schema;
@@ -123,9 +128,9 @@ public class ViewTable
     }
 
     public TranslatableTable apply(List<Object> arguments) {
-      OptiqPrepare.ParseResult parsed =
-          Schemas.parse(MATERIALIZATION_CONNECTION, schema, schemaPath,
-              viewSql);
+      CalcitePrepare.ParseResult parsed =
+          Schemas.parse(MaterializedViewTable.MATERIALIZATION_CONNECTION,
+              schema, schemaPath, viewSql);
       final List<String> schemaPath1 =
           schemaPath != null ? schemaPath : schema.path(null);
       final JavaTypeFactory typeFactory = (JavaTypeFactory) parsed.typeFactory;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/schema/impl/package-info.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/schema/impl/package-info.java b/core/src/main/java/org/apache/calcite/schema/impl/package-info.java
new file mode 100644
index 0000000..8b918de
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/schema/impl/package-info.java
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Utilities to help implement Calcite's SPIs.
+ */
+package org.apache.calcite.schema.impl;
+
+// End package-info.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/schema/package-info.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/schema/package-info.java b/core/src/main/java/org/apache/calcite/schema/package-info.java
new file mode 100644
index 0000000..bded44c
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/schema/package-info.java
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Schema SPI.
+ *
+ * <p>The interfaces in this package define the objects used by the
+ * SQL validator to validate SQL abstract syntax trees and resolve
+ * identifiers to objects.
+ */
+package org.apache.calcite.schema;
+
+// End package-info.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/server/CalciteServer.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/server/CalciteServer.java b/core/src/main/java/org/apache/calcite/server/CalciteServer.java
index f33ffaa..ade037d 100644
--- a/core/src/main/java/org/apache/calcite/server/CalciteServer.java
+++ b/core/src/main/java/org/apache/calcite/server/CalciteServer.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.server;
+package org.apache.calcite.server;
 
 /**
  * Server.
@@ -22,10 +22,10 @@ package net.hydromatic.optiq.server;
  * <p>Represents shared state among connections, and will have monitoring and
  * management facilities.
  */
-public interface OptiqServer {
-  void removeStatement(OptiqServerStatement optiqServerStatement);
+public interface CalciteServer {
+  void removeStatement(CalciteServerStatement calciteServerStatement);
 
-  void addStatement(OptiqServerStatement optiqServerStatement);
+  void addStatement(CalciteServerStatement calciteServerStatement);
 }
 
-// End OptiqServer.java
+// End CalciteServer.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/server/CalciteServerStatement.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/server/CalciteServerStatement.java b/core/src/main/java/org/apache/calcite/server/CalciteServerStatement.java
index 88654d0..61ecf88 100644
--- a/core/src/main/java/org/apache/calcite/server/CalciteServerStatement.java
+++ b/core/src/main/java/org/apache/calcite/server/CalciteServerStatement.java
@@ -14,20 +14,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.server;
+package org.apache.calcite.server;
 
-import net.hydromatic.optiq.jdbc.OptiqConnection;
-import net.hydromatic.optiq.jdbc.OptiqPrepare;
+import org.apache.calcite.jdbc.CalciteConnection;
+import org.apache.calcite.jdbc.CalcitePrepare;
 
 /**
  * Statement within a Calcite server.
  */
-public interface OptiqServerStatement {
+public interface CalciteServerStatement {
   /** Creates a context for preparing a statement for execution. */
-  OptiqPrepare.Context createPrepareContext();
+  CalcitePrepare.Context createPrepareContext();
 
   /** Returns the connection. */
-  OptiqConnection getConnection();
+  CalciteConnection getConnection();
 }
 
-// End OptiqServerStatement.java
+// End CalciteServerStatement.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/server/package-info.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/server/package-info.java b/core/src/main/java/org/apache/calcite/server/package-info.java
index e8dce01..5de3ce7 100644
--- a/core/src/main/java/org/apache/calcite/server/package-info.java
+++ b/core/src/main/java/org/apache/calcite/server/package-info.java
@@ -18,6 +18,6 @@
 /**
  * Provides a server for hosting Calcite connections.
  */
-package net.hydromatic.optiq.server;
+package org.apache.calcite.server;
 
 // End package-info.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/ExplicitOperatorBinding.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/ExplicitOperatorBinding.java b/core/src/main/java/org/apache/calcite/sql/ExplicitOperatorBinding.java
index 38bcf47..0e3bb23 100644
--- a/core/src/main/java/org/apache/calcite/sql/ExplicitOperatorBinding.java
+++ b/core/src/main/java/org/apache/calcite/sql/ExplicitOperatorBinding.java
@@ -14,15 +14,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
+package org.apache.calcite.sql;
 
-import java.util.List;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.runtime.CalciteException;
+import org.apache.calcite.runtime.Resources;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.validate.SqlValidatorException;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.resource.Resources;
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.sql.validate.*;
-import org.eigenbase.util.*;
+import java.util.List;
 
 /**
  * <code>ExplicitOperatorBinding</code> implements {@link SqlOperatorBinding}
@@ -75,7 +76,7 @@ public class ExplicitOperatorBinding extends SqlOperatorBinding {
     return types.get(ordinal);
   }
 
-  public EigenbaseException newError(
+  public CalciteException newError(
       Resources.ExInst<SqlValidatorException> e) {
     if (delegate != null) {
       return delegate.newError(e);

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/JoinConditionType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/JoinConditionType.java b/core/src/main/java/org/apache/calcite/sql/JoinConditionType.java
index 087d17e..6521944 100644
--- a/core/src/main/java/org/apache/calcite/sql/JoinConditionType.java
+++ b/core/src/main/java/org/apache/calcite/sql/JoinConditionType.java
@@ -14,9 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
+package org.apache.calcite.sql;
 
-import org.eigenbase.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.parser.SqlParserPos;
 
 /**
  * Enumerates the types of condition in a join expression.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/JoinType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/JoinType.java b/core/src/main/java/org/apache/calcite/sql/JoinType.java
index b01e3a3..993c279 100644
--- a/core/src/main/java/org/apache/calcite/sql/JoinType.java
+++ b/core/src/main/java/org/apache/calcite/sql/JoinType.java
@@ -14,9 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
+package org.apache.calcite.sql;
 
-import org.eigenbase.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.parser.SqlParserPos;
 
 /**
  * Enumerates the types of join.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/SqlAbstractDateTimeLiteral.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlAbstractDateTimeLiteral.java b/core/src/main/java/org/apache/calcite/sql/SqlAbstractDateTimeLiteral.java
index 5973b4c..0cbf4c3 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlAbstractDateTimeLiteral.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlAbstractDateTimeLiteral.java
@@ -14,14 +14,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
+package org.apache.calcite.sql;
 
-import java.util.*;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.util.ZonelessDate;
+import org.apache.calcite.util.ZonelessTime;
+import org.apache.calcite.util.ZonelessTimestamp;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.util14.*;
+import java.util.Calendar;
+import java.util.TimeZone;
 
 /**
  * A SQL literal representing a DATE, TIME or TIMESTAMP value.
@@ -47,7 +51,8 @@ abstract class SqlAbstractDateTimeLiteral extends SqlLiteral {
    * Constructs a datetime literal based on a Calendar. If the literal is to
    * represent a Timestamp, the Calendar is expected to follow java.sql
    * semantics. If the Calendar is to represent a Time or Date, the Calendar
-   * is expected to follow {@link ZonelessTime} and {@link ZonelessDate}
+   * is expected to follow {@link org.apache.calcite.util.ZonelessTime}
+   * and {@link org.apache.calcite.util.ZonelessDate}
    * semantics.
    */
   protected SqlAbstractDateTimeLiteral(
@@ -114,7 +119,8 @@ abstract class SqlAbstractDateTimeLiteral extends SqlLiteral {
   }
 
   /**
-   * Converts this literal to a {@link ZonelessDate} object.
+   * Converts this literal to a
+   * {@link org.apache.calcite.util.ZonelessDate} object.
    */
   protected ZonelessDate getDate() {
     ZonelessDate zd = new ZonelessDate();
@@ -123,7 +129,8 @@ abstract class SqlAbstractDateTimeLiteral extends SqlLiteral {
   }
 
   /**
-   * Converts this literal to a {@link ZonelessTime} object.
+   * Converts this literal to a
+   * {@link org.apache.calcite.util.ZonelessTime} object.
    */
   protected ZonelessTime getTime() {
     ZonelessTime zt = new ZonelessTime();
@@ -132,7 +139,8 @@ abstract class SqlAbstractDateTimeLiteral extends SqlLiteral {
   }
 
   /**
-   * Converts this literal to a {@link ZonelessTimestamp} object.
+   * Converts this literal to a
+   * {@link org.apache.calcite.util.ZonelessTimestamp} object.
    */
   protected ZonelessTimestamp getTimestamp() {
     ZonelessTimestamp zt = new ZonelessTimestamp();

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/SqlAbstractStringLiteral.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlAbstractStringLiteral.java b/core/src/main/java/org/apache/calcite/sql/SqlAbstractStringLiteral.java
index 7433d53..377479b 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlAbstractStringLiteral.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlAbstractStringLiteral.java
@@ -14,12 +14,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
+package org.apache.calcite.sql;
 
-import java.util.List;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.SqlTypeName;
 
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.sql.type.*;
+import java.util.List;
 
 /**
  * Abstract base for character and binary string literals.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/SqlAccessEnum.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlAccessEnum.java b/core/src/main/java/org/apache/calcite/sql/SqlAccessEnum.java
index d419605..fbbfc25 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlAccessEnum.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlAccessEnum.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
+package org.apache.calcite.sql;
 
 /**
  * Enumeration representing different access types

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/SqlAccessType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlAccessType.java b/core/src/main/java/org/apache/calcite/sql/SqlAccessType.java
index 6940b6c..5da1b56 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlAccessType.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlAccessType.java
@@ -14,9 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
+package org.apache.calcite.sql;
 
-import java.util.*;
+import java.util.EnumSet;
 
 /**
  * SqlAccessType is represented by a set of allowed access types

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/SqlAggFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlAggFunction.java b/core/src/main/java/org/apache/calcite/sql/SqlAggFunction.java
index 8d02a5c..6949d94 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlAggFunction.java
@@ -14,18 +14,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
+package org.apache.calcite.sql;
 
-import org.eigenbase.rel.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.sql.validate.*;
+import org.apache.calcite.sql.type.SqlOperandTypeChecker;
+import org.apache.calcite.sql.type.SqlOperandTypeInference;
+import org.apache.calcite.sql.type.SqlReturnTypeInference;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql.validate.SqlValidatorScope;
 
 /**
  * Abstract base class for the definition of an aggregate function: an operator
  * which aggregates sets of values into a result.
  */
-public abstract class SqlAggFunction extends SqlFunction
-    implements Aggregation {
+public abstract class SqlAggFunction extends SqlFunction {
   //~ Constructors -----------------------------------------------------------
 
   /** Creates a built-in SqlAggFunction. */

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/SqlAsOperator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlAsOperator.java b/core/src/main/java/org/apache/calcite/sql/SqlAsOperator.java
index 371d3b7..1d260cc 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlAsOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlAsOperator.java
@@ -14,17 +14,22 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
+package org.apache.calcite.sql;
 
-import java.util.List;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.type.InferTypes;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.ReturnTypes;
+import org.apache.calcite.sql.util.SqlBasicVisitor;
+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.Util;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.sql.util.*;
-import org.eigenbase.sql.validate.*;
-import org.eigenbase.util.Util;
+import java.util.List;
 
-import static org.eigenbase.util.Static.RESOURCE;
+import static org.apache.calcite.util.Static.RESOURCE;
 
 /**
  * The <code>AS</code> operator associates an expression with an alias.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/SqlBasicCall.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlBasicCall.java b/core/src/main/java/org/apache/calcite/sql/SqlBasicCall.java
index 584d1f8..2e5bf00 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlBasicCall.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlBasicCall.java
@@ -14,12 +14,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
+package org.apache.calcite.sql;
 
-import java.util.List;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.util.UnmodifiableArrayList;
 
-import org.eigenbase.sql.parser.SqlParserPos;
-import org.eigenbase.util.UnmodifiableArrayList;
+import java.util.List;
 
 /**
  * Implementation of {@link SqlCall} that keeps its operands in an array.
@@ -83,8 +83,7 @@ public class SqlBasicCall extends SqlCall {
     return (S) operands[i];
   }
 
-  @Override
-  public int operandCount() {
+  @Override public int operandCount() {
     return operands.length;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 a518ca8..0c0b491 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlBinaryOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlBinaryOperator.java
@@ -14,19 +14,23 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
+package org.apache.calcite.sql;
 
-import java.math.*;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.type.SqlOperandTypeChecker;
+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.SqlMonotonicity;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql.validate.SqlValidatorScope;
+import org.apache.calcite.util.Util;
 
-import java.nio.charset.*;
+import java.math.BigDecimal;
+import java.nio.charset.Charset;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.fun.SqlStdOperatorTable;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.sql.validate.*;
-import org.eigenbase.util.*;
-
-import static org.eigenbase.util.Static.RESOURCE;
+import static org.apache.calcite.util.Static.RESOURCE;
 
 /**
  * <code>SqlBinaryOperator</code> is a binary operator.
@@ -213,8 +217,7 @@ public class SqlBinaryOperator extends SqlOperator {
     return super.getMonotonicity(call, scope);
   }
 
-  @Override
-  public boolean validRexOperands(int count, boolean fail) {
+  @Override public boolean validRexOperands(int count, boolean fail) {
     if (count != 2) {
       // Special exception for AND and OR.
       if ((this == SqlStdOperatorTable.AND

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql/SqlBinaryStringLiteral.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlBinaryStringLiteral.java b/core/src/main/java/org/apache/calcite/sql/SqlBinaryStringLiteral.java
index bf188ad..01ad109 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlBinaryStringLiteral.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlBinaryStringLiteral.java
@@ -14,17 +14,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
+package org.apache.calcite.sql;
 
-import java.util.List;
-
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.util.*;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.util.BitString;
 
 import com.google.common.base.Function;
 import com.google.common.collect.Lists;
 
+import java.util.List;
+
 /**
  * A binary (or hexadecimal) string literal.
  *

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 19cc161..07f5f5b 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlCall.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlCall.java
@@ -14,14 +14,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
-
-import java.util.*;
-
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.sql.util.*;
-import org.eigenbase.sql.validate.*;
+package org.apache.calcite.sql;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.util.SqlVisitor;
+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.SqlValidatorImpl;
+import org.apache.calcite.sql.validate.SqlValidatorScope;
+
+import java.util.ArrayList;
+import java.util.List;
 
 /**
  * A <code>SqlCall</code> is a call to an {@link SqlOperator operator}.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 33b31da..733a360 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlCallBinding.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlCallBinding.java
@@ -14,17 +14,23 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql;
+package org.apache.calcite.sql;
 
-import java.util.*;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.runtime.CalciteException;
+import org.apache.calcite.runtime.Resources;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.validate.SelectScope;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql.validate.SqlValidatorException;
+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.Util;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.resource.Resources;
-import org.eigenbase.sql.fun.*;
-import org.eigenbase.sql.validate.*;
-import org.eigenbase.util.*;
+import java.util.List;
 
-import static org.eigenbase.util.Static.RESOURCE;
+import static org.apache.calcite.util.Static.RESOURCE;
 
 /**
  * <code>SqlCallBinding</code> implements {@link SqlOperatorBinding} by
@@ -166,7 +172,7 @@ public class SqlCallBinding extends SqlOperatorBinding {
     return validator.getParentCursor(paramName);
   }
 
-  public EigenbaseException newError(
+  public CalciteException newError(
       Resources.ExInst<SqlValidatorException> e) {
     return validator.newValidationError(call, e);
   }
@@ -176,7 +182,7 @@ public class SqlCallBinding extends SqlOperatorBinding {
    *
    * @return signature exception
    */
-  public EigenbaseException newValidationSignatureError() {
+  public CalciteException newValidationSignatureError() {
     return validator.newValidationError(call,
         RESOURCE.canNotApplyOp2Type(getOperator().getName(),
             call.getCallSignature(validator, scope),
@@ -190,7 +196,7 @@ public class SqlCallBinding extends SqlOperatorBinding {
    * @param ex underlying exception
    * @return wrapped exception
    */
-  public EigenbaseException newValidationError(
+  public CalciteException newValidationError(
       Resources.ExInst<SqlValidatorException> ex) {
     return validator.newValidationError(call, ex);
   }


[04/58] [abbrv] [partial] incubator-calcite git commit: [CALCITE-306] Standardize code style for "import package.*; "

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/Grouping.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/Grouping.java b/linq4j/src/main/java/org/apache/calcite/linq4j/Grouping.java
index 6226e5a..ea158e9 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/Grouping.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/Grouping.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j;
+package org.apache.calcite.linq4j;
 
 /**
  * Represents a collection of objects that have a common key.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/GroupingImpl.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/GroupingImpl.java b/linq4j/src/main/java/org/apache/calcite/linq4j/GroupingImpl.java
index 8fd4979..de23558 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/GroupingImpl.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/GroupingImpl.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j;
+package org.apache.calcite.linq4j;
 
 import java.util.Collection;
 import java.util.Map;
@@ -35,18 +35,15 @@ class GroupingImpl<K, V> extends AbstractEnumerable<V>
     this.values = values;
   }
 
-  @Override
-  public String toString() {
+  @Override public String toString() {
     return key + ": " + values;
   }
 
-  @Override
-  public int hashCode() {
+  @Override public int hashCode() {
     return (11 + key.hashCode()) * 37 + values.hashCode();
   }
 
-  @Override
-  public boolean equals(Object obj) {
+  @Override public boolean equals(Object obj) {
     return obj instanceof GroupingImpl
            && key.equals(((GroupingImpl) obj).key)
            && values.equals(((GroupingImpl) obj).values);

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/Linq4j.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/Linq4j.java b/linq4j/src/main/java/org/apache/calcite/linq4j/Linq4j.java
index a2c7fda..2b11705 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/Linq4j.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/Linq4j.java
@@ -14,13 +14,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j;
+package org.apache.calcite.linq4j;
 
 import java.io.Closeable;
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
 import java.sql.ResultSet;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
 
 /**
  * Utility and factory methods for Linq4j.
@@ -82,8 +87,8 @@ public abstract class Linq4j {
    * Adapter that converts an enumerator into an iterator.
    *
    * <p><b>WARNING</b>: The iterator returned by this method does not call
-   * {@link net.hydromatic.linq4j.Enumerator#close()}, so it is not safe to use
-   * with an enumerator that allocates resources.</p>
+   * {@link org.apache.calcite.linq4j.Enumerator#close()}, so it is not safe to
+   * use with an enumerator that allocates resources.</p>
    *
    * @param enumerator Enumerator
    * @param <T> Element type
@@ -523,8 +528,7 @@ public abstract class Linq4j {
       return iterable.iterator();
     }
 
-    @Override
-    public boolean any() {
+    @Override public boolean any() {
       return iterable.iterator().hasNext();
     }
   }
@@ -539,23 +543,19 @@ public abstract class Linq4j {
       return (Collection<T>) iterable;
     }
 
-    @Override
-    public int count() {
+    @Override public int count() {
       return getCollection().size();
     }
 
-    @Override
-    public long longCount() {
+    @Override public long longCount() {
       return getCollection().size();
     }
 
-    @Override
-    public boolean contains(T element) {
+    @Override public boolean contains(T element) {
       return getCollection().contains(element);
     }
 
-    @Override
-    public boolean any() {
+    @Override public boolean any() {
       return !getCollection().isEmpty();
     }
   }
@@ -566,13 +566,11 @@ public abstract class Linq4j {
       super(iterable);
     }
 
-    @Override
-    public List<T> toList() {
+    @Override public List<T> toList() {
       return (List<T>) iterable;
     }
 
-    @Override
-    public Enumerable<T> skip(int count) {
+    @Override public Enumerable<T> skip(int count) {
       final List<T> list = toList();
       if (count >= list.size()) {
         return Linq4j.emptyEnumerable();
@@ -580,8 +578,7 @@ public abstract class Linq4j {
       return new ListEnumerable<T>(list.subList(count, list.size()));
     }
 
-    @Override
-    public Enumerable<T> take(int count) {
+    @Override public Enumerable<T> take(int count) {
       final List<T> list = toList();
       if (count >= list.size()) {
         return this;
@@ -589,8 +586,7 @@ public abstract class Linq4j {
       return new ListEnumerable<T>(list.subList(0, count));
     }
 
-    @Override
-    public T elementAt(int index) {
+    @Override public T elementAt(int index) {
       return toList().get(index);
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/Lookup.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/Lookup.java b/linq4j/src/main/java/org/apache/calcite/linq4j/Lookup.java
index d8a59d8..3a4fd96 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/Lookup.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/Lookup.java
@@ -14,9 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j;
+package org.apache.calcite.linq4j;
 
-import net.hydromatic.linq4j.function.Function2;
+import org.apache.calcite.linq4j.function.Function2;
 
 import java.util.Map;
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/LookupImpl.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/LookupImpl.java b/linq4j/src/main/java/org/apache/calcite/linq4j/LookupImpl.java
index d078485..b5bf65c 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/LookupImpl.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/LookupImpl.java
@@ -14,11 +14,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j;
-
-import net.hydromatic.linq4j.function.Function2;
-
-import java.util.*;
+package org.apache.calcite.linq4j;
+
+import org.apache.calcite.linq4j.function.Function2;
+
+import java.util.AbstractCollection;
+import java.util.AbstractMap;
+import java.util.AbstractSet;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
 
 /**
  * Implementation of {@link Lookup} that uses an underlying map.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/OpType.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/OpType.java b/linq4j/src/main/java/org/apache/calcite/linq4j/OpType.java
index 92b3b21..cf12e6c 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/OpType.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/OpType.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j;
+package org.apache.calcite.linq4j;
 
 /**
  * Operator type.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/Ord.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/Ord.java b/linq4j/src/main/java/org/apache/calcite/linq4j/Ord.java
index c919db3..18aa812 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/Ord.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/Ord.java
@@ -14,9 +14,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j;
+package org.apache.calcite.linq4j;
 
-import java.util.*;
+import java.util.AbstractList;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
 
 /**
  * Pair of an element and an ordinal.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/OrderedEnumerable.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/OrderedEnumerable.java b/linq4j/src/main/java/org/apache/calcite/linq4j/OrderedEnumerable.java
index baea12a..0567ea0 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/OrderedEnumerable.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/OrderedEnumerable.java
@@ -14,11 +14,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j;
+package org.apache.calcite.linq4j;
 
 /**
  * Represents the result of applying a sorting operation to an
- * {@link net.hydromatic.linq4j.Enumerable}.
+ * {@link org.apache.calcite.linq4j.Enumerable}.
  *
  * @param <T> element type
  */

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/OrderedQueryable.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/OrderedQueryable.java b/linq4j/src/main/java/org/apache/calcite/linq4j/OrderedQueryable.java
index 8498fe6..c073db4 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/OrderedQueryable.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/OrderedQueryable.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j;
+package org.apache.calcite.linq4j;
 
 /**
  * Represents the result of applying a sorting operation to a {@link Queryable}.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/QueryProvider.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/QueryProvider.java b/linq4j/src/main/java/org/apache/calcite/linq4j/QueryProvider.java
index 6ddf479..08f8734 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/QueryProvider.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/QueryProvider.java
@@ -14,9 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j;
+package org.apache.calcite.linq4j;
 
-import net.hydromatic.linq4j.expressions.Expression;
+import org.apache.calcite.linq4j.tree.Expression;
 
 import java.lang.reflect.Type;
 
@@ -31,7 +31,7 @@ public interface QueryProvider {
    * Constructs a {@link Queryable} object that can evaluate the query
    * represented by a specified expression tree.
    *
-   * <p>NOTE: The {@link net.hydromatic.linq4j.Queryable#getExpression()}
+   * <p>NOTE: The {@link org.apache.calcite.linq4j.Queryable#getExpression()}
    * property of the returned {@link Queryable} object is equal to
    * {@code expression}.</p>
    *
@@ -67,7 +67,7 @@ public interface QueryProvider {
    *
    * <p>The Queryable standard query operator methods that return singleton
    * results call {@code execute}. They pass it a
-   * {@link net.hydromatic.linq4j.expressions.MethodCallExpression}
+   * {@link org.apache.calcite.linq4j.tree.MethodCallExpression}
    * that represents a linq4j query.
    */
   <T> T execute(Expression expression, Class<T> type);

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/QueryProviderImpl.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/QueryProviderImpl.java b/linq4j/src/main/java/org/apache/calcite/linq4j/QueryProviderImpl.java
index d06e800..3c8e50d 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/QueryProviderImpl.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/QueryProviderImpl.java
@@ -14,9 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j;
+package org.apache.calcite.linq4j;
 
-import net.hydromatic.linq4j.expressions.Expression;
+import org.apache.calcite.linq4j.tree.Expression;
 
 import java.lang.reflect.Type;
 
@@ -58,8 +58,7 @@ public abstract class QueryProviderImpl implements QueryProvider {
       super(provider, elementType, expression);
     }
 
-    @Override
-    public String toString() {
+    @Override public String toString() {
       return "Queryable(expr=" + expression + ")";
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/Queryable.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/Queryable.java b/linq4j/src/main/java/org/apache/calcite/linq4j/Queryable.java
index 3e42a57..1a4cd2a 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/Queryable.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/Queryable.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j;
+package org.apache.calcite.linq4j;
 
 /**
  * Provides functionality to evaluate queries against a specific data source

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/QueryableDefaults.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/QueryableDefaults.java b/linq4j/src/main/java/org/apache/calcite/linq4j/QueryableDefaults.java
index e8d1d74..5818b92 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/QueryableDefaults.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/QueryableDefaults.java
@@ -14,16 +14,32 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j;
-
-import net.hydromatic.linq4j.expressions.Expression;
-import net.hydromatic.linq4j.expressions.Expressions;
-import net.hydromatic.linq4j.expressions.FunctionExpression;
-import net.hydromatic.linq4j.function.*;
+package org.apache.calcite.linq4j;
+
+import org.apache.calcite.linq4j.function.BigDecimalFunction1;
+import org.apache.calcite.linq4j.function.DoubleFunction1;
+import org.apache.calcite.linq4j.function.EqualityComparer;
+import org.apache.calcite.linq4j.function.FloatFunction1;
+import org.apache.calcite.linq4j.function.Function1;
+import org.apache.calcite.linq4j.function.Function2;
+import org.apache.calcite.linq4j.function.Functions;
+import org.apache.calcite.linq4j.function.IntegerFunction1;
+import org.apache.calcite.linq4j.function.LongFunction1;
+import org.apache.calcite.linq4j.function.NullableBigDecimalFunction1;
+import org.apache.calcite.linq4j.function.NullableDoubleFunction1;
+import org.apache.calcite.linq4j.function.NullableFloatFunction1;
+import org.apache.calcite.linq4j.function.NullableIntegerFunction1;
+import org.apache.calcite.linq4j.function.NullableLongFunction1;
+import org.apache.calcite.linq4j.function.Predicate1;
+import org.apache.calcite.linq4j.function.Predicate2;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.linq4j.tree.FunctionExpression;
 
 import java.lang.reflect.Type;
 import java.math.BigDecimal;
-import java.util.*;
+import java.util.Comparator;
+import java.util.Iterator;
 
 /**
  * Default implementations for methods in the {@link Queryable} interface.
@@ -405,7 +421,7 @@ public abstract class QueryableDefaults {
    * each group and its key.
    *
    * <p>NOTE: Renamed from {@code groupBy} to distinguish from
-   * {@link #groupBy(net.hydromatic.linq4j.Queryable, net.hydromatic.linq4j.expressions.FunctionExpression, net.hydromatic.linq4j.expressions.FunctionExpression)},
+   * {@link #groupBy(org.apache.calcite.linq4j.Queryable, org.apache.calcite.linq4j.tree.FunctionExpression, org.apache.calcite.linq4j.tree.FunctionExpression)},
    * which has the same erasure.</p>
    */
   public static <T, TKey, TResult> Queryable<Grouping<TKey, TResult>> groupByK(
@@ -436,7 +452,7 @@ public abstract class QueryableDefaults {
    * comparer.
    *
    * <p>NOTE: Renamed from {@code groupBy} to distinguish from
-   * {@link #groupBy(net.hydromatic.linq4j.Queryable, net.hydromatic.linq4j.expressions.FunctionExpression, net.hydromatic.linq4j.expressions.FunctionExpression, net.hydromatic.linq4j.function.EqualityComparer)},
+   * {@link #groupBy(org.apache.calcite.linq4j.Queryable, org.apache.calcite.linq4j.tree.FunctionExpression, org.apache.calcite.linq4j.tree.FunctionExpression, org.apache.calcite.linq4j.function.EqualityComparer)},
    * which has the same erasure.</p>
    */
   public static <T, TKey, TResult> Queryable<TResult> groupByK(
@@ -649,12 +665,13 @@ public abstract class QueryableDefaults {
    * specified type.
    *
    * <p>This method generates a
-   * {@link net.hydromatic.linq4j.expressions.MethodCallExpression} that
+   * {@link org.apache.calcite.linq4j.tree.MethodCallExpression} that
    * represents calling {@code ofType} itself as a constructed generic method.
    * It then passes the {@code MethodCallExpression} to the
-   * {@link net.hydromatic.linq4j.QueryProvider#createQuery createQuery} method of the
-   * {@link net.hydromatic.linq4j.QueryProvider} represented by the Provider property of the source
-   * parameter.</p>
+   * {@link org.apache.calcite.linq4j.QueryProvider#createQuery createQuery}
+   * method of the
+   * {@link org.apache.calcite.linq4j.QueryProvider} represented by
+   * the Provider property of the source parameter.</p>
    *
    * <p>The query behavior that occurs as a result of executing an expression
    * tree that represents calling OfType depends on the implementation of the
@@ -738,7 +755,7 @@ public abstract class QueryableDefaults {
    * form by incorporating the element's index.
    *
    * <p>NOTE: Renamed from {@code select} because had same erasure as
-   * {@link #select(net.hydromatic.linq4j.Queryable, net.hydromatic.linq4j.expressions.FunctionExpression)}.</p>
+   * {@link #select(org.apache.calcite.linq4j.Queryable, org.apache.calcite.linq4j.tree.FunctionExpression)}.
    */
   public static <T, TResult> Queryable<TResult> selectN(Queryable<T> source,
       FunctionExpression<Function2<T, Integer, TResult>> selector) {
@@ -762,7 +779,7 @@ public abstract class QueryableDefaults {
    * projected form of that element.
    *
    * <p>NOTE: Renamed from {@code selectMany} because had same erasure as
-   * {@link #selectMany(net.hydromatic.linq4j.Queryable, net.hydromatic.linq4j.expressions.FunctionExpression)}</p>
+   * {@link #selectMany(org.apache.calcite.linq4j.Queryable, org.apache.calcite.linq4j.tree.FunctionExpression)}.
    */
   public static <T, TResult> Queryable<TResult> selectManyN(Queryable<T> source,
       FunctionExpression<Function2<T, Integer, Enumerable<TResult>>> selector) {
@@ -793,7 +810,7 @@ public abstract class QueryableDefaults {
    * and returned.
    *
    * <p>NOTE: Renamed from {@code selectMany} because had same erasure as
-   * {@link #selectMany(net.hydromatic.linq4j.Queryable, net.hydromatic.linq4j.expressions.FunctionExpression, net.hydromatic.linq4j.expressions.FunctionExpression)}</p>
+   * {@link #selectMany(org.apache.calcite.linq4j.Queryable, org.apache.calcite.linq4j.tree.FunctionExpression, org.apache.calcite.linq4j.tree.FunctionExpression)}.
    */
   public static <T, TCollection, TResult> Queryable<TResult> selectManyN(
       Queryable<T> source,

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/QueryableFactory.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/QueryableFactory.java b/linq4j/src/main/java/org/apache/calcite/linq4j/QueryableFactory.java
index 03549fc..4bb97ec 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/QueryableFactory.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/QueryableFactory.java
@@ -14,10 +14,24 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j;
-
-import net.hydromatic.linq4j.expressions.FunctionExpression;
-import net.hydromatic.linq4j.function.*;
+package org.apache.calcite.linq4j;
+
+import org.apache.calcite.linq4j.function.BigDecimalFunction1;
+import org.apache.calcite.linq4j.function.DoubleFunction1;
+import org.apache.calcite.linq4j.function.EqualityComparer;
+import org.apache.calcite.linq4j.function.FloatFunction1;
+import org.apache.calcite.linq4j.function.Function1;
+import org.apache.calcite.linq4j.function.Function2;
+import org.apache.calcite.linq4j.function.IntegerFunction1;
+import org.apache.calcite.linq4j.function.LongFunction1;
+import org.apache.calcite.linq4j.function.NullableBigDecimalFunction1;
+import org.apache.calcite.linq4j.function.NullableDoubleFunction1;
+import org.apache.calcite.linq4j.function.NullableFloatFunction1;
+import org.apache.calcite.linq4j.function.NullableIntegerFunction1;
+import org.apache.calcite.linq4j.function.NullableLongFunction1;
+import org.apache.calcite.linq4j.function.Predicate1;
+import org.apache.calcite.linq4j.function.Predicate2;
+import org.apache.calcite.linq4j.tree.FunctionExpression;
 
 import java.math.BigDecimal;
 import java.util.Comparator;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/QueryableRecorder.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/QueryableRecorder.java b/linq4j/src/main/java/org/apache/calcite/linq4j/QueryableRecorder.java
index 0fea23e..70e5a1f 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/QueryableRecorder.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/QueryableRecorder.java
@@ -14,21 +14,35 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j;
-
-import net.hydromatic.linq4j.expressions.FunctionExpression;
-import net.hydromatic.linq4j.function.*;
+package org.apache.calcite.linq4j;
+
+import org.apache.calcite.linq4j.function.BigDecimalFunction1;
+import org.apache.calcite.linq4j.function.DoubleFunction1;
+import org.apache.calcite.linq4j.function.EqualityComparer;
+import org.apache.calcite.linq4j.function.FloatFunction1;
+import org.apache.calcite.linq4j.function.Function1;
+import org.apache.calcite.linq4j.function.Function2;
+import org.apache.calcite.linq4j.function.IntegerFunction1;
+import org.apache.calcite.linq4j.function.LongFunction1;
+import org.apache.calcite.linq4j.function.NullableBigDecimalFunction1;
+import org.apache.calcite.linq4j.function.NullableDoubleFunction1;
+import org.apache.calcite.linq4j.function.NullableFloatFunction1;
+import org.apache.calcite.linq4j.function.NullableIntegerFunction1;
+import org.apache.calcite.linq4j.function.NullableLongFunction1;
+import org.apache.calcite.linq4j.function.Predicate1;
+import org.apache.calcite.linq4j.function.Predicate2;
+import org.apache.calcite.linq4j.tree.FunctionExpression;
 
 import java.lang.reflect.Type;
 import java.math.BigDecimal;
 import java.util.Comparator;
 
-import static net.hydromatic.linq4j.QueryableDefaults.NonLeafReplayableQueryable;
+import static org.apache.calcite.linq4j.QueryableDefaults.NonLeafReplayableQueryable;
 
 /**
  * Implementation of {@link QueryableFactory} that records each event
  * and returns an object that can replay the event when you call its
- * {@link net.hydromatic.linq4j.QueryableDefaults.ReplayableQueryable#replay(QueryableFactory)}
+ * {@link org.apache.calcite.linq4j.QueryableDefaults.ReplayableQueryable#replay(QueryableFactory)}
  * method.
  *
  * @param <T> Element type
@@ -659,8 +673,7 @@ public class QueryableRecorder<T> implements QueryableFactory<T> {
         factory.select(source, selector);
       }
 
-      @Override
-      public Type getElementType() {
+      @Override public Type getElementType() {
         return selector.body.type;
       }
     }.castQueryable(); // CHECKSTYLE: IGNORE 0

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/RawEnumerable.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/RawEnumerable.java b/linq4j/src/main/java/org/apache/calcite/linq4j/RawEnumerable.java
index 4c1b78c..75b5de0 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/RawEnumerable.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/RawEnumerable.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j;
+package org.apache.calcite.linq4j;
 
 /**
  * Exposes the enumerator, which supports a simple iteration over a collection,

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/RawQueryable.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/RawQueryable.java b/linq4j/src/main/java/org/apache/calcite/linq4j/RawQueryable.java
index 32238cc..106d5de 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/RawQueryable.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/RawQueryable.java
@@ -14,9 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j;
+package org.apache.calcite.linq4j;
 
-import net.hydromatic.linq4j.expressions.Expression;
+import org.apache.calcite.linq4j.tree.Expression;
 
 import java.lang.reflect.Type;
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/function/BigDecimalFunction1.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/function/BigDecimalFunction1.java b/linq4j/src/main/java/org/apache/calcite/linq4j/function/BigDecimalFunction1.java
index 8778c7c..4f15f67 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/function/BigDecimalFunction1.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/function/BigDecimalFunction1.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.function;
+package org.apache.calcite.linq4j.function;
 
 import java.math.BigDecimal;
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/function/Deterministic.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/function/Deterministic.java b/linq4j/src/main/java/org/apache/calcite/linq4j/function/Deterministic.java
index d08cfa5..469c493 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/function/Deterministic.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/function/Deterministic.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.function;
+package org.apache.calcite.linq4j.function;
 
 import java.lang.annotation.ElementType;
 import java.lang.annotation.Retention;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/function/DoubleFunction1.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/function/DoubleFunction1.java b/linq4j/src/main/java/org/apache/calcite/linq4j/function/DoubleFunction1.java
index 08a9c43..c1d14fe 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/function/DoubleFunction1.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/function/DoubleFunction1.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.function;
+package org.apache.calcite.linq4j.function;
 
 /**
  * Function that takes one parameter and returns a native {@code double} value.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/function/EqualityComparer.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/function/EqualityComparer.java b/linq4j/src/main/java/org/apache/calcite/linq4j/function/EqualityComparer.java
index 8178620..364e998 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/function/EqualityComparer.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/function/EqualityComparer.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.function;
+package org.apache.calcite.linq4j.function;
 
 /**
  * Compares values for equality.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/function/FloatFunction1.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/function/FloatFunction1.java b/linq4j/src/main/java/org/apache/calcite/linq4j/function/FloatFunction1.java
index 3798efb..9e3f785 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/function/FloatFunction1.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/function/FloatFunction1.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.function;
+package org.apache.calcite.linq4j.function;
 
 /**
  * Function that takes one parameter and returns a native {@code float} value.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/function/Function.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/function/Function.java b/linq4j/src/main/java/org/apache/calcite/linq4j/function/Function.java
index d9200a2..53f997c 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/function/Function.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/function/Function.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.function;
+package org.apache.calcite.linq4j.function;
 
 /**
  * Base interface for all functions.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/function/Function0.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/function/Function0.java b/linq4j/src/main/java/org/apache/calcite/linq4j/function/Function0.java
index 4b29ef1..7e1a4cc 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/function/Function0.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/function/Function0.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.function;
+package org.apache.calcite.linq4j.function;
 
 /**
  * Function with no parameters.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/function/Function1.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/function/Function1.java b/linq4j/src/main/java/org/apache/calcite/linq4j/function/Function1.java
index 57490d8..9181a02 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/function/Function1.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/function/Function1.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.function;
+package org.apache.calcite.linq4j.function;
 
 /**
  * Function with one parameter.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/function/Function2.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/function/Function2.java b/linq4j/src/main/java/org/apache/calcite/linq4j/function/Function2.java
index 6509081..5a9f628 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/function/Function2.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/function/Function2.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.function;
+package org.apache.calcite.linq4j.function;
 
 /**
  * Function with two parameters.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/function/Functions.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/function/Functions.java b/linq4j/src/main/java/org/apache/calcite/linq4j/function/Functions.java
index 555a640..12b0a18 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/function/Functions.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/function/Functions.java
@@ -14,14 +14,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.function;
+package org.apache.calcite.linq4j.function;
 
-import net.hydromatic.linq4j.Linq4j;
+import org.apache.calcite.linq4j.Linq4j;
 
 import java.io.Serializable;
 import java.lang.reflect.Type;
 import java.math.BigDecimal;
-import java.util.*;
+import java.util.AbstractList;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
 
 /**
  * Utilities relating to functions.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/function/IntegerFunction1.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/function/IntegerFunction1.java b/linq4j/src/main/java/org/apache/calcite/linq4j/function/IntegerFunction1.java
index b751dd5..2ac526e 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/function/IntegerFunction1.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/function/IntegerFunction1.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.function;
+package org.apache.calcite.linq4j.function;
 
 /**
  * Function that takes one parameter and returns a native {@code int} value.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/function/LongFunction1.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/function/LongFunction1.java b/linq4j/src/main/java/org/apache/calcite/linq4j/function/LongFunction1.java
index bdfbfa0..86b56fe 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/function/LongFunction1.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/function/LongFunction1.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.function;
+package org.apache.calcite.linq4j.function;
 
 /**
  * Function that takes one parameter and returns a native {@code long} value.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/function/NonDeterministic.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/function/NonDeterministic.java b/linq4j/src/main/java/org/apache/calcite/linq4j/function/NonDeterministic.java
index 0d9fd60..6abd894 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/function/NonDeterministic.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/function/NonDeterministic.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.function;
+package org.apache.calcite.linq4j.function;
 
 import java.lang.annotation.ElementType;
 import java.lang.annotation.Retention;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/function/NullableBigDecimalFunction1.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/function/NullableBigDecimalFunction1.java b/linq4j/src/main/java/org/apache/calcite/linq4j/function/NullableBigDecimalFunction1.java
index 10e97ad..fccda8d 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/function/NullableBigDecimalFunction1.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/function/NullableBigDecimalFunction1.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.function;
+package org.apache.calcite.linq4j.function;
 
 import java.math.BigDecimal;
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/function/NullableDoubleFunction1.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/function/NullableDoubleFunction1.java b/linq4j/src/main/java/org/apache/calcite/linq4j/function/NullableDoubleFunction1.java
index 5184d99..57e8f45 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/function/NullableDoubleFunction1.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/function/NullableDoubleFunction1.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.function;
+package org.apache.calcite.linq4j.function;
 
 /**
  * Function that takes one parameter and returns a {@link Double} value that

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/function/NullableFloatFunction1.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/function/NullableFloatFunction1.java b/linq4j/src/main/java/org/apache/calcite/linq4j/function/NullableFloatFunction1.java
index 95529da..0e430de 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/function/NullableFloatFunction1.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/function/NullableFloatFunction1.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.function;
+package org.apache.calcite.linq4j.function;
 
 /**
  * Function that takes one parameter and returns a {@link Float} value that

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/function/NullableIntegerFunction1.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/function/NullableIntegerFunction1.java b/linq4j/src/main/java/org/apache/calcite/linq4j/function/NullableIntegerFunction1.java
index efcd439..e62a946 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/function/NullableIntegerFunction1.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/function/NullableIntegerFunction1.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.function;
+package org.apache.calcite.linq4j.function;
 
 /**
  * Function that takes one parameter and returns an {@link Integer} value that

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/function/NullableLongFunction1.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/function/NullableLongFunction1.java b/linq4j/src/main/java/org/apache/calcite/linq4j/function/NullableLongFunction1.java
index b8acb1e..5be1847 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/function/NullableLongFunction1.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/function/NullableLongFunction1.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.function;
+package org.apache.calcite.linq4j.function;
 
 /**
  * Function that takes one parameter and returns a {@link Long} value that

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/function/Predicate1.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/function/Predicate1.java b/linq4j/src/main/java/org/apache/calcite/linq4j/function/Predicate1.java
index 19a0ae8..062a8f0 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/function/Predicate1.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/function/Predicate1.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.function;
+package org.apache.calcite.linq4j.function;
 
 /**
  * Function with one parameter returning a native {@code boolean} value.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/function/Predicate2.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/function/Predicate2.java b/linq4j/src/main/java/org/apache/calcite/linq4j/function/Predicate2.java
index bd109b5..c0cbe79 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/function/Predicate2.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/function/Predicate2.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.function;
+package org.apache.calcite.linq4j.function;
 
 /**
  * Function with two parameters returning a native {@code boolean} value.
@@ -26,7 +26,7 @@ public interface Predicate2<T0, T1> extends Function<Boolean> {
   /**
    * Predicate that always evaluates to {@code true}.
    *
-   * @see net.hydromatic.linq4j.function.Functions#truePredicate1()
+   * @see org.apache.calcite.linq4j.function.Functions#truePredicate1()
    */
   Predicate2<Object, Object> TRUE = new Predicate2<Object, Object>() {
     public boolean apply(Object v0, Object v1) {
@@ -37,7 +37,7 @@ public interface Predicate2<T0, T1> extends Function<Boolean> {
   /**
    * Predicate that always evaluates to {@code false}.
    *
-   * @see net.hydromatic.linq4j.function.Functions#falsePredicate1()
+   * @see org.apache.calcite.linq4j.function.Functions#falsePredicate1()
    */
   Predicate2<Object, Object> FALSE = new Predicate2<Object, Object>() {
     public boolean apply(Object v0, Object v1) {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/function/package-info.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/function/package-info.java b/linq4j/src/main/java/org/apache/calcite/linq4j/function/package-info.java
index a5ca691..38d9319 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/function/package-info.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/function/package-info.java
@@ -18,6 +18,6 @@
 /**
  * Contains definitions of functions and predicates.
  */
-package net.hydromatic.linq4j.function;
+package org.apache.calcite.linq4j.function;
 
 // End package-info.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/package-info.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/package-info.java b/linq4j/src/main/java/org/apache/calcite/linq4j/package-info.java
index 0c232da..0288428 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/package-info.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/package-info.java
@@ -18,6 +18,6 @@
 /**
  * Language-integrated query for Java (linq4j) main package.
  */
-package net.hydromatic.linq4j;
+package org.apache.calcite.linq4j;
 
 // End package-info.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/tree/AbstractNode.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/AbstractNode.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/AbstractNode.java
index 19edda5..4e73b29 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/AbstractNode.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/AbstractNode.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.expressions;
+package org.apache.calcite.linq4j.tree;
 
 import java.lang.reflect.Type;
 
@@ -45,8 +45,7 @@ public abstract class AbstractNode implements Node {
     return type;
   }
 
-  @Override
-  public String toString() {
+  @Override public String toString() {
     ExpressionWriter writer = new ExpressionWriter(true);
     accept(writer, 0, 0);
     return writer.toString();
@@ -75,8 +74,7 @@ public abstract class AbstractNode implements Node {
         "evaluation not supported: " + getClass() + ":" + nodeType);
   }
 
-  @Override
-  public boolean equals(Object o) {
+  @Override public boolean equals(Object o) {
     if (this == o) {
       return true;
     }
@@ -96,8 +94,7 @@ public abstract class AbstractNode implements Node {
     return true;
   }
 
-  @Override
-  public int hashCode() {
+  @Override public int hashCode() {
     int result = nodeType != null ? nodeType.hashCode() : 0;
     result = 31 * result + (type != null ? type.hashCode() : 0);
     return result;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ArrayLengthRecordField.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ArrayLengthRecordField.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ArrayLengthRecordField.java
index 6bef32a..26cb55f 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ArrayLengthRecordField.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ArrayLengthRecordField.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.expressions;
+package org.apache.calcite.linq4j.tree;
 
 import java.lang.reflect.Array;
 import java.lang.reflect.Type;
@@ -57,8 +57,7 @@ public class ArrayLengthRecordField implements Types.RecordField {
     return clazz;
   }
 
-  @Override
-  public boolean equals(Object o) {
+  @Override public boolean equals(Object o) {
     if (this == o) {
       return true;
     }
@@ -78,10 +77,11 @@ public class ArrayLengthRecordField implements Types.RecordField {
     return true;
   }
 
-  @Override
-  public int hashCode() {
+  @Override public int hashCode() {
     int result = fieldName.hashCode();
     result = 31 * result + clazz.hashCode();
     return result;
   }
 }
+
+// End ArrayLengthRecordField.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/tree/BinaryExpression.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/BinaryExpression.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/BinaryExpression.java
index d614d21..cab48ef 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/BinaryExpression.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/BinaryExpression.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.expressions;
+package org.apache.calcite.linq4j.tree;
 
 import java.lang.reflect.Type;
 
@@ -36,8 +36,7 @@ public class BinaryExpression extends Expression {
     this.primitive = Primitive.of(expression0.getType());
   }
 
-  @Override
-  public Expression accept(Visitor visitor) {
+  @Override public Expression accept(Visitor visitor) {
     visitor = visitor.preVisit(this);
     Expression expression0 = this.expression0.accept(visitor);
     Expression expression1 = this.expression1.accept(visitor);
@@ -161,16 +160,11 @@ public class BinaryExpression extends Expression {
   }
 
   private RuntimeException cannotEvaluate() {
-    return new RuntimeException("cannot evaluate "
-                                + this
-                                + ", nodeType="
-                                + nodeType
-                                + ", primitive="
-                                + primitive);
+    return new RuntimeException("cannot evaluate " + this + ", nodeType="
+      + nodeType + ", primitive=" + primitive);
   }
 
-  @Override
-  public boolean equals(Object o) {
+  @Override public boolean equals(Object o) {
     if (this == o) {
       return true;
     }
@@ -196,8 +190,7 @@ public class BinaryExpression extends Expression {
     return true;
   }
 
-  @Override
-  public int hashCode() {
+  @Override public int hashCode() {
     int result = super.hashCode();
     result = 31 * result + expression0.hashCode();
     result = 31 * result + expression1.hashCode();

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/tree/BlockBuilder.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/BlockBuilder.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/BlockBuilder.java
index fbdcc9b..ef2b44a 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/BlockBuilder.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/BlockBuilder.java
@@ -14,11 +14,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.expressions;
+package org.apache.calcite.linq4j.tree;
 
 import java.lang.reflect.Modifier;
 import java.lang.reflect.Type;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.IdentityHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
 
 /**
  * Builder for {@link BlockStatement}.
@@ -487,8 +493,7 @@ public class BlockBuilder {
       this.map = map;
     }
 
-    @Override
-    public Expression visit(ParameterExpression parameterExpression) {
+    @Override public Expression visit(ParameterExpression parameterExpression) {
       Expression e = map.get(parameterExpression);
       if (e != null) {
         try {
@@ -507,9 +512,8 @@ public class BlockBuilder {
       return super.visit(parameterExpression);
     }
 
-    @Override
-    public Expression visit(UnaryExpression unaryExpression, Expression
-        expression) {
+    @Override public Expression visit(UnaryExpression unaryExpression,
+        Expression expression) {
       if (unaryExpression.getNodeType().modifiesLvalue) {
         expression = unaryExpression.expression; // avoid substitution
         if (expression instanceof ParameterExpression) {
@@ -544,8 +548,7 @@ public class BlockBuilder {
     private final Map<ParameterExpression, Slot> map =
         new IdentityHashMap<ParameterExpression, Slot>();
 
-    @Override
-    public Expression visit(ParameterExpression parameter) {
+    @Override public Expression visit(ParameterExpression parameter) {
       final Slot slot = map.get(parameter);
       if (slot != null) {
         // Count use of parameter, if it's registered. It's OK if

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/tree/BlockStatement.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/BlockStatement.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/BlockStatement.java
index a6c697f..364afeb 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/BlockStatement.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/BlockStatement.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.expressions;
+package org.apache.calcite.linq4j.tree;
 
 import java.lang.reflect.Type;
 import java.util.HashSet;
@@ -53,16 +53,14 @@ public class BlockStatement extends Statement {
     return true;
   }
 
-  @Override
-  public BlockStatement accept(Visitor visitor) {
+  @Override public BlockStatement accept(Visitor visitor) {
     visitor = visitor.preVisit(this);
     List<Statement> newStatements = Expressions.acceptStatements(statements,
         visitor);
     return visitor.visit(this, newStatements);
   }
 
-  @Override
-  void accept0(ExpressionWriter writer) {
+  @Override void accept0(ExpressionWriter writer) {
     if (statements.isEmpty()) {
       writer.append("{}");
       return;
@@ -74,8 +72,7 @@ public class BlockStatement extends Statement {
     writer.end("}\n");
   }
 
-  @Override
-  public Object evaluate(Evaluator evaluator) {
+  @Override public Object evaluate(Evaluator evaluator) {
     Object o = null;
     for (Statement statement : statements) {
       o = statement.evaluate(evaluator);
@@ -83,8 +80,7 @@ public class BlockStatement extends Statement {
     return o;
   }
 
-  @Override
-  public boolean equals(Object o) {
+  @Override public boolean equals(Object o) {
     if (this == o) {
       return true;
     }
@@ -104,8 +100,7 @@ public class BlockStatement extends Statement {
     return true;
   }
 
-  @Override
-  public int hashCode() {
+  @Override public int hashCode() {
     int result = hash;
     if (result == 0) {
       result = super.hashCode();

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/tree/Blocks.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/Blocks.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/Blocks.java
index 7b73794..103abd7 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/Blocks.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/Blocks.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.expressions;
+package org.apache.calcite.linq4j.tree;
 
 /**
  * <p>Helper methods concerning {@link BlockStatement}s.</p>
@@ -40,10 +40,8 @@ public final class Blocks {
         statement = Expressions.return_(null, (Expression) body);
       }
     } else {
-      throw new AssertionError(
-          "block cannot contain node that is neither statement nor "
-          + "expression: "
-          + body);
+      throw new AssertionError("block cannot contain node that is neither "
+          + "statement nor expression: " + body);
     }
     return Expressions.block(statement);
   }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/tree/CallSiteBinder.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/CallSiteBinder.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/CallSiteBinder.java
index ab6cb97..6f14b7d 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/CallSiteBinder.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/CallSiteBinder.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.expressions;
+package org.apache.calcite.linq4j.tree;
 
 /**
  * Creates a {@link DynamicExpression} that represents a dynamic operation bound

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/tree/CatchBlock.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/CatchBlock.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/CatchBlock.java
index 43078df..e134c41 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/CatchBlock.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/CatchBlock.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.expressions;
+package org.apache.calcite.linq4j.tree;
 
 /**
  * Represents a catch statement in a try block.
@@ -29,8 +29,7 @@ public class CatchBlock {
     this.body = body;
   }
 
-  @Override
-  public boolean equals(Object o) {
+  @Override public boolean equals(Object o) {
     if (this == o) {
       return true;
     }
@@ -51,8 +50,7 @@ public class CatchBlock {
     return true;
   }
 
-  @Override
-  public int hashCode() {
+  @Override public int hashCode() {
     int result = parameter != null ? parameter.hashCode() : 0;
     result = 31 * result + (body != null ? body.hashCode() : 0);
     return result;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ClassDeclaration.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ClassDeclaration.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ClassDeclaration.java
index e8ed2e2..e11a2e8 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ClassDeclaration.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ClassDeclaration.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.expressions;
+package org.apache.calcite.linq4j.tree;
 
 import java.lang.reflect.Modifier;
 import java.lang.reflect.Type;
@@ -65,8 +65,7 @@ public class ClassDeclaration extends MemberDeclaration {
     return visitor.visit(this, members1);
   }
 
-  @Override
-  public boolean equals(Object o) {
+  @Override public boolean equals(Object o) {
     if (this == o) {
       return true;
     }
@@ -101,8 +100,7 @@ public class ClassDeclaration extends MemberDeclaration {
     return true;
   }
 
-  @Override
-  public int hashCode() {
+  @Override public int hashCode() {
     int result = modifier;
     result = 31 * result + classClass.hashCode();
     result = 31 * result + name.hashCode();

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ClassDeclarationFinder.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ClassDeclarationFinder.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ClassDeclarationFinder.java
index ce5ed85..2388538 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ClassDeclarationFinder.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ClassDeclarationFinder.java
@@ -14,9 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.expressions;
+package org.apache.calcite.linq4j.tree;
 
-import net.hydromatic.linq4j.function.Function1;
+import org.apache.calcite.linq4j.function.Function1;
 
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
@@ -144,8 +144,7 @@ public class ClassDeclarationFinder extends Visitor {
    * @param newExpression expression to optimize
    * @return nested visitor if anonymous class is given
    */
-  @Override
-  public Visitor preVisit(NewExpression newExpression) {
+  @Override public Visitor preVisit(NewExpression newExpression) {
     if (newExpression.memberDeclarations == null) {
       return this;
     }
@@ -160,15 +159,13 @@ public class ClassDeclarationFinder extends Visitor {
    * @param classDeclaration expression to optimize
    * @return nested visitor
    */
-  @Override
-  public Visitor preVisit(ClassDeclaration classDeclaration) {
+  @Override public Visitor preVisit(ClassDeclaration classDeclaration) {
     ClassDeclarationFinder visitor = goDeeper();
     visitor.learnFinalStaticDeclarations(classDeclaration.memberDeclarations);
     return visitor;
   }
 
-  @Override
-  public Expression visit(NewExpression newExpression,
+  @Override public Expression visit(NewExpression newExpression,
       List<Expression> arguments, List<MemberDeclaration> memberDeclarations) {
     if (parent == null) {
       // Unable to optimize since no wrapper class exists to put fields to.
@@ -212,8 +209,7 @@ public class ClassDeclarationFinder extends Visitor {
     return newExpression;
   }
 
-  @Override
-  public ClassDeclaration visit(ClassDeclaration classDeclaration,
+  @Override public ClassDeclaration visit(ClassDeclaration classDeclaration,
       List<MemberDeclaration> memberDeclarations) {
     memberDeclarations = optimizeDeclarations(memberDeclarations);
     return super.visit(classDeclaration, memberDeclarations);

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ConditionalExpression.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ConditionalExpression.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ConditionalExpression.java
index 4462c3e..da5e60a 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ConditionalExpression.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ConditionalExpression.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.expressions;
+package org.apache.calcite.linq4j.tree;
 
 import java.lang.reflect.Type;
 import java.util.List;
@@ -40,8 +40,7 @@ public class ConditionalExpression extends AbstractNode {
     this.expressionList = expressionList;
   }
 
-  @Override
-  void accept(ExpressionWriter writer, int lprec, int rprec) {
+  @Override void accept(ExpressionWriter writer, int lprec, int rprec) {
     for (int i = 0; i < expressionList.size(); i += 2) {
       writer.append(i > 0 ? " else if (" : "if (")
           .append(expressionList.get(i))
@@ -54,8 +53,7 @@ public class ConditionalExpression extends AbstractNode {
     }
   }
 
-  @Override
-  public boolean equals(Object o) {
+  @Override public boolean equals(Object o) {
     if (this == o) {
       return true;
     }
@@ -75,8 +73,7 @@ public class ConditionalExpression extends AbstractNode {
     return true;
   }
 
-  @Override
-  public int hashCode() {
+  @Override public int hashCode() {
     int result = super.hashCode();
     result = 31 * result + expressionList.hashCode();
     return result;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ConditionalStatement.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ConditionalStatement.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ConditionalStatement.java
index 4e51e3e..c280dac 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ConditionalStatement.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ConditionalStatement.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.expressions;
+package org.apache.calcite.linq4j.tree;
 
 import java.util.List;
 
@@ -39,15 +39,13 @@ public class ConditionalStatement extends Statement {
     this.expressionList = expressionList;
   }
 
-  @Override
-  public Statement accept(Visitor visitor) {
+  @Override public Statement accept(Visitor visitor) {
     visitor = visitor.preVisit(this);
     List<Node> list = Expressions.acceptNodes(expressionList, visitor);
     return visitor.visit(this, list);
   }
 
-  @Override
-  void accept0(ExpressionWriter writer) {
+  @Override void accept0(ExpressionWriter writer) {
     for (int i = 0; i < expressionList.size() - 1; i += 2) {
       if (i > 0) {
         writer.backUp();
@@ -69,8 +67,7 @@ public class ConditionalStatement extends Statement {
     return collection.get(collection.size() - 1);
   }
 
-  @Override
-  public boolean equals(Object o) {
+  @Override public boolean equals(Object o) {
     if (this == o) {
       return true;
     }
@@ -90,8 +87,7 @@ public class ConditionalStatement extends Statement {
     return true;
   }
 
-  @Override
-  public int hashCode() {
+  @Override public int hashCode() {
     int result = super.hashCode();
     result = 31 * result + expressionList.hashCode();
     return result;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ConstantExpression.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ConstantExpression.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ConstantExpression.java
index 701ac43..dcc1962 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ConstantExpression.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ConstantExpression.java
@@ -14,10 +14,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.expressions;
+package org.apache.calcite.linq4j.tree;
 
-import net.hydromatic.linq4j.function.Function1;
-import net.hydromatic.linq4j.function.Functions;
+import org.apache.calcite.linq4j.function.Function1;
+import org.apache.calcite.linq4j.function.Functions;
 
 import java.lang.reflect.Constructor;
 import java.lang.reflect.Field;
@@ -54,13 +54,11 @@ public class ConstantExpression extends Expression {
     return value;
   }
 
-  @Override
-  public Expression accept(Visitor visitor) {
+  @Override public Expression accept(Visitor visitor) {
     return visitor.visit(this);
   }
 
-  @Override
-  void accept(ExpressionWriter writer, int lprec, int rprec) {
+  @Override void accept(ExpressionWriter writer, int lprec, int rprec) {
     if (value == null) {
       if (!writer.requireParentheses(this, lprec, rprec)) {
         writer.append("(").append(type).append(") null");
@@ -236,8 +234,7 @@ public class ConstantExpression extends Expression {
     buf.append('"');
   }
 
-  @Override
-  public boolean equals(Object o) {
+  @Override public boolean equals(Object o) {
     // REVIEW: Should constants with the same value and different type
     // (e.g. 3L and 3) be considered equal.
     if (this == o) {
@@ -259,8 +256,7 @@ public class ConstantExpression extends Expression {
     return true;
   }
 
-  @Override
-  public int hashCode() {
+  @Override public int hashCode() {
     int result = super.hashCode();
     result = 31 * result + (value != null ? value.hashCode() : 0);
     return result;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ConstantUntypedNull.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ConstantUntypedNull.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ConstantUntypedNull.java
index 2831987..7fa36fe 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ConstantUntypedNull.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ConstantUntypedNull.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.expressions;
+package org.apache.calcite.linq4j.tree;
 
 /**
  * Represents a constant null of unknown type
@@ -32,18 +32,17 @@ public class ConstantUntypedNull extends ConstantExpression {
     super(Object.class, null);
   }
 
-  @Override
-  void accept(ExpressionWriter writer, int lprec, int rprec) {
+  @Override void accept(ExpressionWriter writer, int lprec, int rprec) {
     writer.append("null");
   }
 
-  @Override
-  public boolean equals(Object o) {
+  @Override public boolean equals(Object o) {
     return o == INSTANCE;
   }
 
-  @Override
-  public int hashCode() {
+  @Override public int hashCode() {
     return ConstantUntypedNull.class.hashCode();
   }
 }
+
+// End ConstantUntypedNull.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ConstructorDeclaration.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ConstructorDeclaration.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ConstructorDeclaration.java
index ac96ee0..95b4f1c 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ConstructorDeclaration.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ConstructorDeclaration.java
@@ -14,10 +14,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.expressions;
+package org.apache.calcite.linq4j.tree;
 
-import net.hydromatic.linq4j.function.Function1;
-import net.hydromatic.linq4j.function.Functions;
+import org.apache.calcite.linq4j.function.Function1;
+import org.apache.calcite.linq4j.function.Functions;
 
 import java.lang.reflect.Modifier;
 import java.lang.reflect.Type;
@@ -47,8 +47,7 @@ public class ConstructorDeclaration extends MemberDeclaration {
     this.body = body;
   }
 
-  @Override
-  public MemberDeclaration accept(Visitor visitor) {
+  @Override public MemberDeclaration accept(Visitor visitor) {
     visitor = visitor.preVisit(this);
     // do not visit parameters
     final BlockStatement body = this.body.accept(visitor);
@@ -78,8 +77,7 @@ public class ConstructorDeclaration extends MemberDeclaration {
     writer.newlineAndIndent();
   }
 
-  @Override
-  public boolean equals(Object o) {
+  @Override public boolean equals(Object o) {
     if (this == o) {
       return true;
     }
@@ -105,8 +103,7 @@ public class ConstructorDeclaration extends MemberDeclaration {
     return true;
   }
 
-  @Override
-  public int hashCode() {
+  @Override public int hashCode() {
     int result = hash;
     if (result == 0) {
       result = modifier;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/tree/DeclarationStatement.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/DeclarationStatement.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/DeclarationStatement.java
index 2c96c4e..c861f41 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/DeclarationStatement.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/DeclarationStatement.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.expressions;
+package org.apache.calcite.linq4j.tree;
 
 import java.lang.reflect.Modifier;
 
@@ -35,8 +35,7 @@ public class DeclarationStatement extends Statement {
     this.initializer = initializer;
   }
 
-  @Override
-  public DeclarationStatement accept(Visitor visitor) {
+  @Override public DeclarationStatement accept(Visitor visitor) {
     visitor = visitor.preVisit(this);
     // do not visit parameter - visit may not return a ParameterExpression
     Expression initializer = this.initializer != null
@@ -45,8 +44,7 @@ public class DeclarationStatement extends Statement {
     return visitor.visit(this, initializer);
   }
 
-  @Override
-  void accept0(ExpressionWriter writer) {
+  @Override void accept0(ExpressionWriter writer) {
     final String modifiers = Modifier.toString(this.modifiers);
     if (!modifiers.isEmpty()) {
       writer.append(modifiers).append(' ');
@@ -75,8 +73,7 @@ public class DeclarationStatement extends Statement {
     }
   }
 
-  @Override
-  public boolean equals(Object o) {
+  @Override public boolean equals(Object o) {
     if (this == o) {
       return true;
     }
@@ -103,8 +100,7 @@ public class DeclarationStatement extends Statement {
     return true;
   }
 
-  @Override
-  public int hashCode() {
+  @Override public int hashCode() {
     int result = super.hashCode();
     result = 31 * result + modifiers;
     result = 31 * result + parameter.hashCode();

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/tree/DefaultExpression.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/DefaultExpression.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/DefaultExpression.java
index 576d8ca..8569528 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/DefaultExpression.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/DefaultExpression.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.expressions;
+package org.apache.calcite.linq4j.tree;
 
 /**
  * Represents the default value of a type or an empty expression.
@@ -24,8 +24,7 @@ public class DefaultExpression extends Expression {
     super(ExpressionType.Default, type);
   }
 
-  @Override
-  public Expression accept(Visitor visitor) {
+  @Override public Expression accept(Visitor visitor) {
     return visitor.visit(this);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/tree/DeterministicCodeOptimizer.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/DeterministicCodeOptimizer.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/DeterministicCodeOptimizer.java
index 0fd3087..f161c43 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/DeterministicCodeOptimizer.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/DeterministicCodeOptimizer.java
@@ -14,16 +14,22 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.expressions;
+package org.apache.calcite.linq4j.tree;
 
-import net.hydromatic.linq4j.function.Deterministic;
-import net.hydromatic.linq4j.function.NonDeterministic;
+import org.apache.calcite.linq4j.function.Deterministic;
+import org.apache.calcite.linq4j.function.NonDeterministic;
 
 import java.lang.reflect.Method;
 import java.lang.reflect.Modifier;
 import java.math.BigDecimal;
 import java.math.BigInteger;
-import java.util.*;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.IdentityHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
 import java.util.regex.Pattern;
 
 /**
@@ -86,8 +92,8 @@ public class DeterministicCodeOptimizer extends ClassDeclarationFinder {
    * @param newExpression expression to optimize
    * @return optimized expression
    */
-  @Override
-  protected Expression tryOptimizeNewInstance(NewExpression newExpression) {
+  @Override protected Expression
+  tryOptimizeNewInstance(NewExpression newExpression) {
     if (newExpression.type instanceof Class
         && isConstant(newExpression.arguments)
         && isConstructorDeterministic(newExpression)) {
@@ -97,8 +103,7 @@ public class DeterministicCodeOptimizer extends ClassDeclarationFinder {
     return newExpression;
   }
 
-  @Override
-  public Expression visit(BinaryExpression binaryExpression,
+  @Override public Expression visit(BinaryExpression binaryExpression,
       Expression expression0, Expression expression1) {
     Expression result = super.visit(binaryExpression, expression0, expression1);
     if (binaryExpression.getNodeType().modifiesLvalue) {
@@ -111,8 +116,7 @@ public class DeterministicCodeOptimizer extends ClassDeclarationFinder {
     return result;
   }
 
-  @Override
-  public Expression visit(TernaryExpression ternaryExpression,
+  @Override public Expression visit(TernaryExpression ternaryExpression,
       Expression expression0, Expression expression1, Expression expression2) {
     Expression result =
         super.visit(ternaryExpression, expression0, expression1, expression2);
@@ -125,8 +129,7 @@ public class DeterministicCodeOptimizer extends ClassDeclarationFinder {
     return result;
   }
 
-  @Override
-  public Expression visit(UnaryExpression unaryExpression,
+  @Override public Expression visit(UnaryExpression unaryExpression,
       Expression expression) {
     Expression result = super.visit(unaryExpression, expression);
 
@@ -139,8 +142,7 @@ public class DeterministicCodeOptimizer extends ClassDeclarationFinder {
     return result;
   }
 
-  @Override
-  public Expression visit(TypeBinaryExpression typeBinaryExpression,
+  @Override public Expression visit(TypeBinaryExpression typeBinaryExpression,
       Expression expression) {
     Expression result = super.visit(typeBinaryExpression, expression);
 
@@ -166,8 +168,7 @@ public class DeterministicCodeOptimizer extends ClassDeclarationFinder {
     return methodCallExpression;
   }
 
-  @Override
-  public Expression visit(MethodCallExpression methodCallExpression,
+  @Override public Expression visit(MethodCallExpression methodCallExpression,
       Expression targetExpression, List<Expression> expressions) {
     Expression result =
         super.visit(methodCallExpression, targetExpression, expressions);
@@ -176,8 +177,7 @@ public class DeterministicCodeOptimizer extends ClassDeclarationFinder {
     return result;
   }
 
-  @Override
-  public Expression visit(MemberExpression memberExpression,
+  @Override public Expression visit(MemberExpression memberExpression,
       Expression expression) {
     Expression result = super.visit(memberExpression, expression);
 
@@ -188,8 +188,7 @@ public class DeterministicCodeOptimizer extends ClassDeclarationFinder {
     return result;
   }
 
-  @Override
-  public MemberDeclaration visit(FieldDeclaration fieldDeclaration,
+  @Override public MemberDeclaration visit(FieldDeclaration fieldDeclaration,
       Expression initializer) {
     if (Modifier.isStatic(fieldDeclaration.modifier)) {
       // Avoid optimization of static fields, since we'll have to track order
@@ -205,8 +204,7 @@ public class DeterministicCodeOptimizer extends ClassDeclarationFinder {
    *
    * @param memberDeclarations list of declarations to search finals from
    */
-  @Override
-  protected void learnFinalStaticDeclarations(
+  @Override protected void learnFinalStaticDeclarations(
       List<MemberDeclaration> memberDeclarations) {
     for (MemberDeclaration decl : memberDeclarations) {
       if (decl instanceof FieldDeclaration) {
@@ -298,8 +296,7 @@ public class DeterministicCodeOptimizer extends ClassDeclarationFinder {
    * @param expression expression to test
    * @return true when the expression is known to be constant
    */
-  @Override
-  protected boolean isConstant(Expression expression) {
+  @Override protected boolean isConstant(Expression expression) {
     return expression == null
         || expression instanceof ConstantExpression
         || !constants.isEmpty() && constants.containsKey(expression)
@@ -351,8 +348,7 @@ public class DeterministicCodeOptimizer extends ClassDeclarationFinder {
    * @param name name of the variable to test
    * @return true if the name is used by one of static final fields
    */
-  @Override
-  protected boolean hasField(String name) {
+  @Override protected boolean hasField(String name) {
     return !fieldsByName.isEmpty() && fieldsByName.containsKey(name)
         || parent != null && parent.hasField(name);
   }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/tree/DynamicExpression.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/DynamicExpression.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/DynamicExpression.java
index d0a2a7e..ff03001 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/DynamicExpression.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/DynamicExpression.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.expressions;
+package org.apache.calcite.linq4j.tree;
 
 /**
  * Represents a dynamic operation.
@@ -24,8 +24,7 @@ public class DynamicExpression extends Expression {
     super(ExpressionType.Dynamic, type);
   }
 
-  @Override
-  public Expression accept(Visitor visitor) {
+  @Override public Expression accept(Visitor visitor) {
     return visitor.visit(this);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ElementInit.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ElementInit.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ElementInit.java
index eb51973..de080ce 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ElementInit.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ElementInit.java
@@ -14,11 +14,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.expressions;
+package org.apache.calcite.linq4j.tree;
 
 /**
  * Represents an initializer for a single element of an
- * {@link net.hydromatic.linq4j.Enumerable} collection.
+ * {@link org.apache.calcite.linq4j.Enumerable} collection.
  */
 public class ElementInit {
 }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/tree/Evaluator.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/Evaluator.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/Evaluator.java
index a63fa40..9508bbd 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/Evaluator.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/Evaluator.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.expressions;
+package org.apache.calcite.linq4j.tree;
 
 import java.util.ArrayList;
 import java.util.List;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/tree/Expression.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/Expression.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/Expression.java
index 91f3610..4c3427b 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/Expression.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/Expression.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.expressions;
+package org.apache.calcite.linq4j.tree;
 
 import java.lang.reflect.Type;
 
@@ -40,8 +40,7 @@ public abstract class Expression extends AbstractNode {
     assert type != null;
   }
 
-  @Override
-  // More specific return type.
+  @Override // More specific return type.
   public abstract Expression accept(Visitor visitor);
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ExpressionType.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ExpressionType.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ExpressionType.java
index 396a63b..fcdf3a5 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ExpressionType.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ExpressionType.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.linq4j.expressions;
+package org.apache.calcite.linq4j.tree;
 
 /**
  * <p>Analogous to LINQ's System.Linq.Expressions.ExpressionType.</p>


[57/58] [abbrv] incubator-calcite git commit: [CALCITE-370] Support GROUPING SETS, CUBE, ROLLUP in SQL and algebra

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6d79b5eb/core/src/main/java/org/apache/calcite/sql2rel/RelDecorrelator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql2rel/RelDecorrelator.java b/core/src/main/java/org/apache/calcite/sql2rel/RelDecorrelator.java
index 6da2268..5f2ef57 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/RelDecorrelator.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/RelDecorrelator.java
@@ -32,6 +32,7 @@ import org.apache.calcite.rel.RelCollation;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.RelShuttleImpl;
 import org.apache.calcite.rel.RelVisitor;
+import org.apache.calcite.rel.core.Aggregate;
 import org.apache.calcite.rel.core.AggregateCall;
 import org.apache.calcite.rel.core.Correlation;
 import org.apache.calcite.rel.core.Correlator;
@@ -427,6 +428,9 @@ public class RelDecorrelator implements ReflectiveVisitor {
    * @param rel Aggregate to rewrite
    */
   public void decorrelateRel(LogicalAggregate rel) {
+    if (rel.getGroupType() != Aggregate.Group.SIMPLE) {
+      throw new AssertionError(Bug.CALCITE_461_FIXED);
+    }
     //
     // Rewrite logic:
     //
@@ -578,11 +582,8 @@ public class RelDecorrelator implements ReflectiveVisitor {
     }
 
     LogicalAggregate newAggregate =
-        new LogicalAggregate(
-            rel.getCluster(),
-            newProjectRel,
-            ImmutableBitSet.range(newGroupKeyCount),
-            newAggCalls);
+        new LogicalAggregate(rel.getCluster(), newProjectRel, false,
+            ImmutableBitSet.range(newGroupKeyCount), null, newAggCalls);
 
     mapOldToNewRel.put(rel, newAggregate);
 
@@ -2003,15 +2004,11 @@ public class RelDecorrelator implements ReflectiveVisitor {
       extends RelOptRule {
     public RemoveCorrelationForScalarAggregateRule() {
       super(
-          operand(
-              Correlator.class,
+          operand(Correlator.class,
               operand(RelNode.class, any()),
-              operand(
-                  LogicalProject.class,
-                  operand(
-                      LogicalAggregate.class,
-                      operand(
-                          LogicalProject.class,
+              operand(LogicalProject.class,
+                  operand(LogicalAggregate.class, null, Aggregate.IS_SIMPLE,
+                      operand(LogicalProject.class,
                           operand(RelNode.class, any()))))));
     }
 
@@ -2347,11 +2344,8 @@ public class RelDecorrelator implements ReflectiveVisitor {
       ImmutableBitSet groupSet =
           ImmutableBitSet.range(groupCount);
       LogicalAggregate newAggRel =
-          new LogicalAggregate(
-              cluster,
-              joinOutputProjRel,
-              groupSet,
-              newAggCalls);
+          new LogicalAggregate(cluster, joinOutputProjRel, false, groupSet,
+              null, newAggCalls);
 
       List<RexNode> newAggOutputProjExprList = Lists.newArrayList();
       for (int i : groupSet) {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6d79b5eb/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 c68273d..8885444 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/RelFieldTrimmer.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/RelFieldTrimmer.java
@@ -58,8 +58,10 @@ import org.apache.calcite.util.mapping.Mapping;
 import org.apache.calcite.util.mapping.MappingType;
 import org.apache.calcite.util.mapping.Mappings;
 
+import com.google.common.base.Function;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
 
 import java.math.BigDecimal;
 import java.util.ArrayList;
@@ -734,6 +736,9 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
       Aggregate aggregate,
       ImmutableBitSet fieldsUsed,
       Set<RelDataTypeField> extraFields) {
+    if (aggregate.indicator) {
+      throw new AssertionError(Bug.CALCITE_461_FIXED);
+    }
     // Fields:
     //
     // | sys fields | group fields | agg functions |
@@ -798,6 +803,16 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
     final ImmutableBitSet newGroupSet =
         Mappings.apply(inputMapping, aggregate.getGroupSet());
 
+    final ImmutableList<ImmutableBitSet> newGroupSets =
+        ImmutableList.copyOf(
+            Iterables.transform(
+                aggregate.getGroupSets(),
+                new Function<ImmutableBitSet, ImmutableBitSet>() {
+                  public ImmutableBitSet apply(ImmutableBitSet input) {
+                    return Mappings.apply(inputMapping, input);
+                  }
+                }));
+
     // Populate mapping of where to find the fields. System and grouping
     // fields first.
     for (IntPair pair : inputMapping) {
@@ -827,7 +842,7 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
     }
 
     RelNode newAggregate = aggregateFactory.createAggregate(newInput,
-        newGroupSet, newAggCallList);
+        false, newGroupSet, newGroupSets, newAggCallList);
 
     assert newAggregate.getClass() == aggregate.getClass();
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6d79b5eb/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java b/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
index ceb6a1a..c7a3d0a 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
@@ -16,6 +16,7 @@
  */
 package org.apache.calcite.sql2rel;
 
+import org.apache.calcite.linq4j.Linq4j;
 import org.apache.calcite.linq4j.Ord;
 import org.apache.calcite.plan.Convention;
 import org.apache.calcite.plan.RelOptCluster;
@@ -148,6 +149,7 @@ import java.lang.reflect.Type;
 import java.math.BigDecimal;
 import java.util.AbstractList;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.EnumSet;
@@ -710,10 +712,10 @@ public class SqlToRelConverter {
 
     // Usual case: all of the expressions in the SELECT clause are
     // different.
+    final ImmutableBitSet groupSet =
+        ImmutableBitSet.range(rel.getRowType().getFieldCount());
     rel =
-        createAggregate(
-            bb,
-            ImmutableBitSet.range(rel.getRowType().getFieldCount()),
+        createAggregate(bb, false, groupSet, ImmutableList.of(groupSet),
             ImmutableList.<AggregateCall>of());
 
     bb.setRoot(
@@ -1032,7 +1034,8 @@ public class SqlToRelConverter {
         final int keyCount = leftKeys.size();
         final List<Integer> args = ImmutableIntList.range(0, keyCount);
         LogicalAggregate aggregate =
-            new LogicalAggregate(cluster, seek, ImmutableBitSet.of(),
+            new LogicalAggregate(cluster, seek, false, ImmutableBitSet.of(),
+                null,
                 ImmutableList.of(
                     new AggregateCall(SqlStdOperatorTable.COUNT, false,
                         ImmutableList.<Integer>of(), longType, null),
@@ -2531,63 +2534,28 @@ public class SqlToRelConverter {
     // Currently farrago allows expressions, not just column references in
     // group by list. This is not SQL 2003 compliant.
 
-    Map<Integer, Integer> groupExprProjection =
-        new HashMap<Integer, Integer>();
+    final Map<Integer, Integer> groupExprProjection = Maps.newHashMap();
 
-    int i = -1;
+    final ImmutableList.Builder<ImmutableList<ImmutableBitSet>> builder =
+        ImmutableList.builder();
     for (SqlNode groupExpr : groupList) {
-      ++i;
-      final SqlNode expandedGroupExpr =
-          validator.expand(groupExpr, bb.scope);
-      aggConverter.addGroupExpr(expandedGroupExpr);
-
-      if (expandedGroupExpr instanceof SqlIdentifier) {
-        // SQL 2003 does not allow expressions of column references
-        SqlIdentifier expr = (SqlIdentifier) expandedGroupExpr;
-
-        // column references should be fully qualified.
-        assert expr.names.size() == 2;
-        String originalRelName = expr.names.get(0);
-        String originalFieldName = expr.names.get(1);
-
-        int[] nsIndexes = {-1};
-        final SqlValidatorScope[] ancestorScopes = {null};
-        SqlValidatorNamespace foundNs =
-            bb.scope.resolve(
-                originalRelName,
-                ancestorScopes,
-                nsIndexes);
-
-        assert foundNs != null;
-        assert nsIndexes.length == 1;
-        int childNamespaceIndex = nsIndexes[0];
-
-        int namespaceOffset = 0;
-
-        if (childNamespaceIndex > 0) {
-          // If not the first child, need to figure out the width of
-          // output types from all the preceding namespaces
-          assert ancestorScopes[0] instanceof ListScope;
-          List<SqlValidatorNamespace> children =
-              ((ListScope) ancestorScopes[0]).getChildren();
-
-          for (int j = 0; j < childNamespaceIndex; j++) {
-            namespaceOffset +=
-                children.get(j).getRowType().getFieldCount();
-          }
-        }
+      convertGroupItem(bb, aggConverter, groupExprProjection, builder,
+          groupExpr);
+    }
 
-        RelDataTypeField field =
-            catalogReader.field(foundNs.getRowType(), originalFieldName);
-        int origPos = namespaceOffset + field.getIndex();
+    final Set<ImmutableBitSet> flatGroupSets =
+        Sets.newTreeSet(ImmutableBitSet.COMPARATOR);
+    for (List<ImmutableBitSet> groupSet : Linq4j.product(builder.build())) {
+      flatGroupSets.add(ImmutableBitSet.union(groupSet));
+    }
 
-        groupExprProjection.put(origPos, i);
-      }
+    // For GROUP BY (), we need a singleton grouping set.
+    if (flatGroupSets.isEmpty()) {
+      flatGroupSets.add(ImmutableBitSet.of());
     }
 
     RexNode havingExpr = null;
-    List<RexNode> selectExprs = new ArrayList<RexNode>();
-    List<String> selectNames = new ArrayList<String>();
+    final List<Pair<RexNode, String>> projects = Lists.newArrayList();
 
     try {
       Util.permAssert(bb.agg == null, "already in agg mode");
@@ -2643,10 +2611,9 @@ public class SqlToRelConverter {
 
       // Add the aggregator
       bb.setRoot(
-          createAggregate(
-              bb,
+          createAggregate(bb, false,
               ImmutableBitSet.range(aggConverter.groupExprs.size()),
-              aggConverter.getAggCalls()),
+              ImmutableList.copyOf(flatGroupSets), aggConverter.getAggCalls()),
           false);
 
       bb.mapRootRelToFieldProjection.put(bb.root, groupExprProjection);
@@ -2687,16 +2654,17 @@ public class SqlToRelConverter {
           selectNamespace.getRowType().getFieldNames();
       int sysFieldCount = selectList.size() - names.size();
       for (SqlNode expr : selectList) {
-        selectExprs.add(bb.convertExpression(expr));
-        selectNames.add(
-            k < sysFieldCount
-                ? validator.deriveAlias(expr, k++)
-                : names.get(k++ - sysFieldCount));
+        projects.add(
+            Pair.of(bb.convertExpression(expr),
+                k < sysFieldCount
+                    ? validator.deriveAlias(expr, k++)
+                    : names.get(k++ - sysFieldCount)));
       }
 
       for (SqlNode expr : orderExprList) {
-        selectExprs.add(bb.convertExpression(expr));
-        selectNames.add(validator.deriveAlias(expr, k++));
+        projects.add(
+            Pair.of(bb.convertExpression(expr),
+                validator.deriveAlias(expr, k++)));
       }
     } finally {
       bb.agg = null;
@@ -2711,8 +2679,7 @@ public class SqlToRelConverter {
     bb.setRoot(
         RelOptUtil.createProject(
             bb.root,
-            selectExprs,
-            selectNames,
+            projects,
             true),
         false);
 
@@ -2724,6 +2691,175 @@ public class SqlToRelConverter {
     }
   }
 
+  private void convertGroupItem(Blackboard bb, AggConverter aggConverter,
+      Map<Integer, Integer> groupExprProjection,
+      ImmutableList.Builder<ImmutableList<ImmutableBitSet>> topBuilder,
+      SqlNode groupExpr) {
+    final ImmutableList.Builder<ImmutableBitSet> builder =
+        ImmutableList.builder();
+    switch (groupExpr.getKind()) {
+    case GROUPING_SETS:
+      convertGroupSet(bb, aggConverter, groupExprProjection, builder,
+            groupExpr);
+      topBuilder.add(builder.build());
+      return;
+    case CUBE:
+    case ROLLUP:
+      // E.g. ROLLUP(a, (b, c)) becomes [{0}, {1, 2}]
+      // then we roll up to [(0, 1, 2), (0), ()]  -- note no (0, 1)
+      List<ImmutableBitSet> bitSets =
+          convertGroupTuple(bb, aggConverter,
+              groupExprProjection, ((SqlCall) groupExpr).getOperandList());
+      switch (groupExpr.getKind()) {
+      case ROLLUP:
+        rollup(builder, bitSets);
+        break;
+      default:
+        cube(builder, bitSets);
+        break;
+      }
+      topBuilder.add(builder.build());
+      return;
+    case OTHER:
+      if (groupExpr instanceof SqlNodeList) {
+        SqlNodeList list = (SqlNodeList) groupExpr;
+        for (SqlNode node : list) {
+          convertGroupItem(bb, aggConverter, groupExprProjection, topBuilder,
+              node);
+        }
+        return;
+      }
+      // fall through
+    default:
+      convertGroupSet(bb, aggConverter, groupExprProjection, builder,
+          groupExpr);
+      topBuilder.add(builder.build());
+    }
+  }
+
+  private void rollup(ImmutableList.Builder<ImmutableBitSet> builder,
+      List<ImmutableBitSet> bitSets) {
+    for (;;) {
+      builder.add(ImmutableBitSet.union(bitSets));
+      if (bitSets.isEmpty()) {
+        break;
+      }
+      bitSets = bitSets.subList(0, bitSets.size() - 1);
+    }
+  }
+
+  private void cube(ImmutableList.Builder<ImmutableBitSet> builder,
+      List<ImmutableBitSet> bitSets) {
+    // Given the bit sets [{1}, {2, 3}, {5}],
+    // form the lists [[{1}, {}], [{2, 3}, {}], [{5}, {}]].
+    final List<List<ImmutableBitSet>> bits = Lists.newArrayList();
+    for (ImmutableBitSet bitSet : bitSets) {
+      bits.add(Arrays.asList(bitSet, ImmutableBitSet.of()));
+    }
+    for (List<ImmutableBitSet> o : Linq4j.product(bits)) {
+      builder.add(ImmutableBitSet.union(o));
+    }
+  }
+
+  private void convertGroupSet(Blackboard bb, AggConverter aggConverter,
+      Map<Integer, Integer> groupExprProjection,
+      ImmutableList.Builder<ImmutableBitSet> builder, SqlNode groupExpr) {
+    switch (groupExpr.getKind()) {
+    case GROUPING_SETS:
+      final SqlCall call = (SqlCall) groupExpr;
+      for (SqlNode node : call.getOperandList()) {
+        convertGroupSet(bb, aggConverter, groupExprProjection, builder, node);
+      }
+      return;
+    case ROW:
+      final List<ImmutableBitSet> bitSets =
+          convertGroupTuple(bb, aggConverter, groupExprProjection,
+              ((SqlCall) groupExpr).getOperandList());
+      builder.add(ImmutableBitSet.union(bitSets));
+      return;
+    default:
+      builder.add(
+          convertGroupExpr(bb, aggConverter, groupExprProjection, groupExpr));
+      return;
+    }
+  }
+
+  private List<ImmutableBitSet> convertGroupTuple(Blackboard bb,
+      AggConverter aggConverter, Map<Integer, Integer> groupExprProjection,
+      List<SqlNode> operandList) {
+    List<ImmutableBitSet> list = Lists.newArrayList();
+    for (SqlNode operand : operandList) {
+      list.add(
+          convertGroupExpr(bb, aggConverter, groupExprProjection, operand));
+    }
+    return list;
+  }
+
+  private ImmutableBitSet convertGroupExpr(Blackboard bb,
+      AggConverter aggConverter, Map<Integer, Integer> groupExprProjection,
+      SqlNode groupExpr) {
+    final SqlNode expandedGroupExpr =
+        validator.expand(groupExpr, bb.scope);
+
+    switch (expandedGroupExpr.getKind()) {
+    case ROW:
+      return ImmutableBitSet.union(
+          convertGroupTuple(bb, aggConverter, groupExprProjection,
+              ((SqlCall) expandedGroupExpr).getOperandList()));
+    case OTHER:
+      if (expandedGroupExpr instanceof SqlNodeList
+          && ((SqlNodeList) expandedGroupExpr).size() == 0) {
+        return ImmutableBitSet.of();
+      }
+    }
+
+    final int ref = aggConverter.addGroupExpr(expandedGroupExpr);
+    if (expandedGroupExpr instanceof SqlIdentifier) {
+      // SQL 2003 does not allow expressions of column references
+      SqlIdentifier expr = (SqlIdentifier) expandedGroupExpr;
+
+      // column references should be fully qualified.
+      assert expr.names.size() == 2;
+      String originalRelName = expr.names.get(0);
+      String originalFieldName = expr.names.get(1);
+
+      int[] nsIndexes = {-1};
+      final SqlValidatorScope[] ancestorScopes = {null};
+      SqlValidatorNamespace foundNs =
+          bb.scope.resolve(
+              originalRelName,
+              ancestorScopes,
+              nsIndexes);
+
+      assert foundNs != null;
+      assert nsIndexes.length == 1;
+      int childNamespaceIndex = nsIndexes[0];
+
+      int namespaceOffset = 0;
+
+      if (childNamespaceIndex > 0) {
+        // If not the first child, need to figure out the width of
+        // output types from all the preceding namespaces
+        assert ancestorScopes[0] instanceof ListScope;
+        List<SqlValidatorNamespace> children =
+            ((ListScope) ancestorScopes[0]).getChildren();
+
+        for (int j = 0; j < childNamespaceIndex; j++) {
+          namespaceOffset +=
+              children.get(j).getRowType().getFieldCount();
+        }
+      }
+
+      RelDataTypeField field =
+          catalogReader.field(foundNs.getRowType(), originalFieldName);
+      int origPos = namespaceOffset + field.getIndex();
+
+      groupExprProjection.put(origPos, ref);
+    }
+
+    return ImmutableBitSet.of(ref);
+  }
+
   /**
    * Creates an Aggregate.
    *
@@ -2737,18 +2873,21 @@ public class SqlToRelConverter {
    * parameter.
    *
    * @param bb       Blackboard
+   * @param indicator Whether to output fields indicating grouping sets
    * @param groupSet Bit set of ordinals of grouping columns
+   * @param groupSets Grouping sets
    * @param aggCalls Array of calls to aggregate functions
    * @return LogicalAggregate
    */
-  protected RelNode createAggregate(
-      Blackboard bb,
-      ImmutableBitSet groupSet,
+  protected RelNode createAggregate(Blackboard bb, boolean indicator,
+      ImmutableBitSet groupSet, ImmutableList<ImmutableBitSet> groupSets,
       List<AggregateCall> aggCalls) {
     return new LogicalAggregate(
         cluster,
         bb.root,
+        indicator,
         groupSet,
+        groupSets,
         aggCalls);
   }
 
@@ -4068,18 +4207,17 @@ public class SqlToRelConverter {
       }
     }
 
-    // implement SqlRexContext
     public RexNode convertExpression(SqlNode expr) {
       // If we're in aggregation mode and this is an expression in the
       // GROUP BY clause, return a reference to the field.
       if (agg != null) {
         final SqlNode expandedGroupExpr = validator.expand(expr, scope);
-        RexNode rex = agg.lookupGroupExpr(expandedGroupExpr);
-        if (rex != null) {
-          return rex;
+        final RexInputRef ref = agg.lookupGroupExpr(expandedGroupExpr);
+        if (ref != null) {
+          return ref;
         }
         if (expr instanceof SqlCall) {
-          rex = agg.lookupAggregates((SqlCall) expr);
+          final RexNode rex = agg.lookupAggregates((SqlCall) expr);
           if (rex != null) {
             return rex;
           }
@@ -4449,17 +4587,19 @@ public class SqlToRelConverter {
       }
     }
 
-    public void addGroupExpr(SqlNode expr) {
+    public int addGroupExpr(SqlNode expr) {
       RexNode convExpr = bb.convertExpression(expr);
-      final RexNode rex = lookupGroupExpr(expr);
-      if (rex != null) {
-        return; // don't add duplicates, in e.g. "GROUP BY x, y, x"
+      RexInputRef ref = lookupGroupExpr(expr);
+      if (ref == null) {
+        // Don't add duplicates, in e.g. "GROUP BY x, y, x"
+        groupExprs.add(expr);
+        String name = nameMap.get(expr.toString());
+        addExpr(convExpr, name);
+        final RelDataType type = convExpr.getType();
+        ref = rexBuilder.makeInputRef(type, inputRefs.size());
+        inputRefs.add(ref);
       }
-      groupExprs.add(expr);
-      String name = nameMap.get(expr.toString());
-      addExpr(convExpr, name);
-      final RelDataType type = convExpr.getType();
-      inputRefs.add(rexBuilder.makeInputRef(type, inputRefs.size()));
+      return ref.getIndex();
     }
 
     /**
@@ -4608,7 +4748,7 @@ public class SqlToRelConverter {
      * expressions, returns a reference to the expression, otherwise returns
      * null.
      */
-    public RexNode lookupGroupExpr(SqlNode expr) {
+    public RexInputRef lookupGroupExpr(SqlNode expr) {
       for (int i = 0; i < groupExprs.size(); i++) {
         SqlNode groupExpr = groupExprs.get(i);
         if (expr.equalsDeep(groupExpr, false)) {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6d79b5eb/core/src/main/java/org/apache/calcite/util/Bug.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/Bug.java b/core/src/main/java/org/apache/calcite/util/Bug.java
index b851f27..6d9db47 100644
--- a/core/src/main/java/org/apache/calcite/util/Bug.java
+++ b/core/src/main/java/org/apache/calcite/util/Bug.java
@@ -168,6 +168,11 @@ public abstract class Bug {
    * Pull up filters rejected by a ProjectableFilterableTable</a> is fixed. */
   public static final boolean CALCITE_445_FIXED = false;
 
+  /** Whether
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-461">[CALCITE-461]
+   * Convert more planner rules to handle grouping sets</a> is fixed. */
+  public static final boolean CALCITE_461_FIXED = false;
+
   /**
    * Use this to flag temporary code.
    */

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6d79b5eb/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java b/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
index 20ecfa5..178ffc9 100644
--- a/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
+++ b/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
@@ -22,6 +22,7 @@ import org.apache.calcite.adapter.jdbc.JdbcSchema;
 import org.apache.calcite.interpreter.Row;
 import org.apache.calcite.linq4j.AbstractEnumerable;
 import org.apache.calcite.linq4j.Enumerable;
+import org.apache.calcite.linq4j.EnumerableDefaults;
 import org.apache.calcite.linq4j.Enumerator;
 import org.apache.calcite.linq4j.ExtendedEnumerable;
 import org.apache.calcite.linq4j.Linq4j;
@@ -128,6 +129,9 @@ public enum BuiltInMethod {
   GROUP_BY(ExtendedEnumerable.class, "groupBy", Function1.class),
   GROUP_BY2(ExtendedEnumerable.class, "groupBy", Function1.class,
       Function0.class, Function2.class, Function2.class),
+  GROUP_BY_MULTIPLE(EnumerableDefaults.class, "groupByMultiple",
+      Enumerable.class, List.class, Function0.class, Function2.class,
+      Function2.class),
   AGGREGATE(ExtendedEnumerable.class, "aggregate", Object.class,
       Function2.class, Function1.class),
   ORDER_BY(ExtendedEnumerable.class, "orderBy", Function1.class,

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6d79b5eb/core/src/main/java/org/apache/calcite/util/Util.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/Util.java b/core/src/main/java/org/apache/calcite/util/Util.java
index 435df11..a100315 100644
--- a/core/src/main/java/org/apache/calcite/util/Util.java
+++ b/core/src/main/java/org/apache/calcite/util/Util.java
@@ -27,6 +27,7 @@ import com.google.common.cache.LoadingCache;
 import com.google.common.collect.Collections2;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Ordering;
 
 import java.io.BufferedInputStream;
 import java.io.BufferedOutputStream;
@@ -55,6 +56,7 @@ import java.util.AbstractSet;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.Comparator;
 import java.util.Enumeration;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -1742,9 +1744,20 @@ public class Util {
     }
   }
 
-  /** Returns whether an iterable is in ascending order. */
+  /** Returns whether an iterable is in ascending order.
+   *
+   * <p>This implies that it contains no duplicates. */
   public static <E extends Comparable<E>>
   boolean isStrictlySorted(Iterable<? extends E> list) {
+    return isStrictlySorted(list, Ordering.natural());
+  }
+
+  /** Returns whether an iterable is in ascending order, according to a
+   * {@link java.util.Comparator}.
+   *
+   * <p>This implies that it contains no duplicates. */
+  public static <E> boolean isStrictlySorted(Iterable<? extends E> list,
+      Comparator<E> comparator) {
     final Iterator<? extends E> iterator = list.iterator();
     if (!iterator.hasNext()) {
       return true;
@@ -1755,7 +1768,7 @@ public class Util {
         return true;
       }
       E next = iterator.next();
-      if (e.compareTo(next) >= 0) {
+      if (comparator.compare(e, next) >= 0) {
         return false;
       }
       e = next;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6d79b5eb/core/src/main/java/org/apache/calcite/util/mapping/Mappings.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/mapping/Mappings.java b/core/src/main/java/org/apache/calcite/util/mapping/Mappings.java
index 8c16163..385df5b 100644
--- a/core/src/main/java/org/apache/calcite/util/mapping/Mappings.java
+++ b/core/src/main/java/org/apache/calcite/util/mapping/Mappings.java
@@ -204,10 +204,7 @@ public abstract class Mappings {
       final int target = mapping.getTarget(source);
       builder.set(target);
     }
-    if (builder.wouldEqual(bitSet)) {
-      return bitSet;
-    }
-    return builder.build();
+    return builder.build(bitSet);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6d79b5eb/core/src/test/java/org/apache/calcite/plan/RelWriterTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/plan/RelWriterTest.java b/core/src/test/java/org/apache/calcite/plan/RelWriterTest.java
index 6dc6aa4..eaaf2a6 100644
--- a/core/src/test/java/org/apache/calcite/plan/RelWriterTest.java
+++ b/core/src/test/java/org/apache/calcite/plan/RelWriterTest.java
@@ -133,12 +133,11 @@ public class RelWriterTest {
                                 "deptno", true),
                             rexBuilder.makeExactLiteral(BigDecimal.TEN)));
                 final RelJsonWriter writer = new RelJsonWriter();
-                final RelDataType intType =
-                    cluster.getTypeFactory().createSqlType(SqlTypeName.INTEGER);
                 final RelDataType bigIntType =
                     cluster.getTypeFactory().createSqlType(SqlTypeName.BIGINT);
                 LogicalAggregate aggregate =
-                    new LogicalAggregate(cluster, filter, ImmutableBitSet.of(0),
+                    new LogicalAggregate(cluster, filter, false,
+                        ImmutableBitSet.of(0), null,
                         ImmutableList.of(
                             new AggregateCall(SqlStdOperatorTable.COUNT,
                                 true, ImmutableList.of(1), bigIntType, "c"),

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6d79b5eb/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java b/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
index b6f6318..3f754ba 100644
--- a/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
+++ b/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
@@ -76,7 +76,11 @@ public class SqlParserTest {
   protected void check(
       String sql,
       String expected) {
-    getTester().check(sql, expected);
+    sql(sql).ok(expected);
+  }
+
+  protected Sql sql(String sql) {
+    return new Sql(sql);
   }
 
   private SqlParser getSqlParser(String sql) {
@@ -109,7 +113,7 @@ public class SqlParserTest {
   protected void checkFails(
       String sql,
       String expectedMsgPattern) {
-    getTester().checkFails(sql, expectedMsgPattern);
+    sql(sql).fails(expectedMsgPattern);
   }
 
   /**
@@ -189,8 +193,7 @@ public class SqlParserTest {
   }
 
   @Test public void testColumnAliasWithoutAs() {
-    check(
-        "select 1 foo from emp",
+    check("select 1 foo from emp",
         "SELECT 1 AS `FOO`\n"
             + "FROM `EMP`");
   }
@@ -204,9 +207,7 @@ public class SqlParserTest {
   }
 
   @Test public void testEmbeddedTimestamp() {
-    checkExp(
-        "{ts '1998-10-22 16:22:34'}",
-        "TIMESTAMP '1998-10-22 16:22:34'");
+    checkExp("{ts '1998-10-22 16:22:34'}", "TIMESTAMP '1998-10-22 16:22:34'");
   }
 
   @Test public void testNot() {
@@ -261,14 +262,12 @@ public class SqlParserTest {
   }
 
   @Test public void testIsBooleanPrecedenceAndAssociativity() {
-    check(
-        "select * from t where x is unknown is not unknown",
+    check("select * from t where x is unknown is not unknown",
         "SELECT *\n"
             + "FROM `T`\n"
             + "WHERE ((`X` IS UNKNOWN) IS NOT UNKNOWN)");
 
-    check(
-        "select 1 from t where not true is unknown",
+    check("select 1 from t where not true is unknown",
         "SELECT 1\n"
             + "FROM `T`\n"
             + "WHERE ((NOT TRUE) IS UNKNOWN)");
@@ -292,8 +291,7 @@ public class SqlParserTest {
     checkExp("'abc'=123", "('abc' = 123)");
     checkExp("'abc'<>123", "('abc' <> 123)");
     checkExp("'abc'<>123='def'<>456", "((('abc' <> 123) = 'def') <> 456)");
-    checkExp(
-        "'abc'<>123=('def'<>456)", "(('abc' <> 123) = ('def' <> 456))");
+    checkExp("'abc'<>123=('def'<>456)", "(('abc' <> 123) = ('def' <> 456))");
   }
 
   @Test public void testBangEqualIsBad() {
@@ -304,8 +302,7 @@ public class SqlParserTest {
     //   equals' is <> as in BASIC. There are many texts which will tell
     //   you that != is SQL's not-equals operator; those texts are false;
     //   it's one of those unstampoutable urban myths."
-    checkFails(
-        "'abc'^!^=123",
+    checkFails("'abc'^!^=123",
         "Lexical error at line 1, column 6\\.  Encountered: \"!\" \\(33\\), after : \"\"");
   }
 
@@ -654,8 +651,7 @@ public class SqlParserTest {
   }
 
   @Test public void testFromWithAs() {
-    check(
-        "select 1 from emp as e where 1",
+    check("select 1 from emp as e where 1",
         "SELECT 1\n"
             + "FROM `EMP` AS `E`\n"
             + "WHERE 1");
@@ -684,8 +680,7 @@ public class SqlParserTest {
   }
 
   @Test public void testFunction() {
-    check(
-        "select substring('Eggs and ham', 1, 3 + 2) || ' benedict' from emp",
+    check("select substring('Eggs and ham', 1, 3 + 2) || ' benedict' from emp",
         "SELECT (SUBSTRING('Eggs and ham' FROM 1 FOR (3 + 2)) || ' benedict')\n"
             + "FROM `EMP`");
     checkExp(
@@ -697,8 +692,7 @@ public class SqlParserTest {
     checkExp("count(DISTINCT 1)", "COUNT(DISTINCT 1)");
     checkExp("count(ALL 1)", "COUNT(ALL 1)");
     checkExp("count(1)", "COUNT(1)");
-    check(
-        "select count(1), count(distinct 2) from emp",
+    check("select count(1), count(distinct 2) from emp",
         "SELECT COUNT(1), COUNT(DISTINCT 2)\n"
             + "FROM `EMP`");
   }
@@ -730,13 +724,17 @@ public class SqlParserTest {
             + "HAVING (1 = 2)\n"
             + "ORDER BY 3");
 
-    checkFails(
-        "select 1 from emp group by ()^,^ x",
-        "(?s)Encountered \\\",\\\" at .*");
+    // Used to be invalid, valid now that we support grouping sets.
+    sql("select 1 from emp group by (), x")
+        .ok("SELECT 1\n"
+            + "FROM `EMP`\n"
+            + "GROUP BY (), `X`");
 
-    checkFails(
-        "select 1 from emp group by x, ^(^)",
-        "(?s)Encountered \"\\( \\)\" at .*");
+    // Used to be invalid, valid now that we support grouping sets.
+    sql("select 1 from emp group by x, ()")
+        .ok("SELECT 1\n"
+            + "FROM `EMP`\n"
+            + "GROUP BY `X`, ()");
 
     // parentheses do not an empty GROUP BY make
     check(
@@ -770,6 +768,69 @@ public class SqlParserTest {
             + "HAVING (COUNT(*) > 5)");
   }
 
+  @Test public void testGroupingSets() {
+    sql("select deptno from emp\n"
+        + "group by grouping sets (deptno, (deptno, gender), ())")
+        .ok("SELECT `DEPTNO`\n"
+            + "FROM `EMP`\n"
+            + "GROUP BY (GROUPING_SETS(`DEPTNO`, (ROW(`DEPTNO`, `GENDER`)),))");
+
+    // Grouping sets must have parentheses
+    sql("select deptno from emp\n"
+        + "group by grouping sets ^deptno^, (deptno, gender), ()")
+        .fails("(?s).*Encountered \"deptno\" at line 2, column 24.\n"
+            + "Was expecting:\n"
+            + "    \"\\(\" .*");
+
+    // Nested grouping sets, cube, rollup, grouping sets all OK
+    sql("select deptno from emp\n"
+        + "group by grouping sets (deptno, grouping sets (e, d), (),\n"
+        + "  cube (x, y), rollup(p, q))\n"
+        + "order by a")
+        .ok("SELECT `DEPTNO`\n"
+            + "FROM `EMP`\n"
+            + "GROUP BY (GROUPING_SETS(`DEPTNO`, (GROUPING_SETS(`E`, `D`)),, (CUBE(`X`, `Y`)), (ROLLUP(`P`, `Q`))))\n"
+            + "ORDER BY `A`");
+
+    sql("select deptno from emp\n"
+        + "group by grouping sets (())")
+        .ok("SELECT `DEPTNO`\n"
+            + "FROM `EMP`\n"
+            + "GROUP BY (GROUPING_SETS())");
+  }
+
+  @Test public void testGroupByCube() {
+    sql("select deptno from emp\n"
+        + "group by cube ((a, b), (c, d))")
+        .ok("SELECT `DEPTNO`\n"
+            + "FROM `EMP`\n"
+            + "GROUP BY (CUBE((ROW(`A`, `B`)), (ROW(`C`, `D`))))");
+  }
+
+  @Test public void testGroupByCube2() {
+    sql("select deptno from emp\n"
+        + "group by cube ((a, b), (c, d)) order by a")
+        .ok("SELECT `DEPTNO`\n"
+            + "FROM `EMP`\n"
+            + "GROUP BY (CUBE((ROW(`A`, `B`)), (ROW(`C`, `D`))))\n"
+            + "ORDER BY `A`");
+    sql("select deptno from emp\n"
+        + "group by cube (^)")
+        .fails("(?s)Encountered \"\\)\" at .*");
+  }
+
+  @Test public void testGroupByRollup() {
+    sql("select deptno from emp\n"
+        + "group by rollup (deptno, deptno + 1, gender)")
+        .ok("SELECT `DEPTNO`\n" + "FROM `EMP`\n"
+            + "GROUP BY (ROLLUP(`DEPTNO`, (`DEPTNO` + 1), `GENDER`))");
+
+    // Nested rollup not ok
+    sql("select deptno from emp\n"
+        + "group by rollup (deptno^, rollup(e, d))")
+        .fails("(?s)Encountered \", rollup\" at .*");
+  }
+
   @Test public void testWith() {
     check(
         "with femaleEmps as (select * from emps where gender = 'F')"
@@ -5733,6 +5794,24 @@ public class SqlParserTest {
       // Do nothing. We're not interested in unparsing invalid SQL
     }
   }
+
+  /** Helper class for building fluent code such as
+   * {@code sql("values 1").ok();}. */
+  private class Sql {
+    private final String sql;
+
+    Sql(String sql) {
+      this.sql = sql;
+    }
+
+    public void ok(String expected) {
+      getTester().check(sql, expected);
+    }
+
+    public void fails(String expectedMsgPattern) {
+      getTester().checkFails(sql, expectedMsgPattern);
+    }
+  }
 }
 
 // End SqlParserTest.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6d79b5eb/core/src/test/java/org/apache/calcite/test/FoodmartTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/FoodmartTest.java b/core/src/test/java/org/apache/calcite/test/FoodmartTest.java
index a8999f1..311da93 100644
--- a/core/src/test/java/org/apache/calcite/test/FoodmartTest.java
+++ b/core/src/test/java/org/apache/calcite/test/FoodmartTest.java
@@ -111,7 +111,7 @@ public class FoodmartTest {
     if (!CalciteAssert.ENABLE_SLOW && idList == null) {
       // Avoid loading the query set in a regular test suite run. It burns too
       // much memory.
-      return ImmutableList.of();
+      return ImmutableList.of(new Object[] {-1});
     }
     final FoodMartQuerySet set = FoodMartQuerySet.instance();
     final List<Object[]> list = new ArrayList<Object[]>();
@@ -146,7 +146,13 @@ public class FoodmartTest {
   }
 
   public FoodmartTest(int id) throws IOException {
-    this.query = FoodMartQuerySet.instance().queries.get(id);
+    if (id < 0) {
+      this.query = new FoodmartQuery();
+      query.id = id;
+      query.sql = "select * from (values 1) as t(c)";
+    } else {
+      this.query = FoodMartQuerySet.instance().queries.get(id);
+    }
     assert query.id == id : id + ":" + query.id;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6d79b5eb/core/src/test/java/org/apache/calcite/test/JdbcTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/JdbcTest.java b/core/src/test/java/org/apache/calcite/test/JdbcTest.java
index 291dff1..61f6d51 100644
--- a/core/src/test/java/org/apache/calcite/test/JdbcTest.java
+++ b/core/src/test/java/org/apache/calcite/test/JdbcTest.java
@@ -2866,6 +2866,30 @@ public class JdbcTest {
             "deptno=10; commission=250; S=11500.0");
   }
 
+  @Test public void testGroupingSets() {
+    CalciteAssert.that()
+        .with(CalciteAssert.Config.REGULAR)
+        .query("select \"deptno\", count(*) as c, sum(\"salary\") as s\n"
+            + "from \"hr\".\"emps\"\n"
+            + "group by grouping sets((\"deptno\"), ())")
+        .returnsUnordered(
+            "deptno=0; C=4; S=36500.0",
+            "deptno=10; C=3; S=28500.0",
+            "deptno=20; C=1; S=8000.0");
+  }
+
+  @Test public void testRollup() {
+    CalciteAssert.that()
+        .with(CalciteAssert.Config.REGULAR)
+        .query("select \"deptno\", count(*) as c, sum(\"salary\") as s\n"
+            + "from \"hr\".\"emps\"\n"
+            + "group by rollup(\"deptno\")")
+        .returnsUnordered(
+            "deptno=0; C=4; S=36500.0",
+            "deptno=10; C=3; S=28500.0",
+            "deptno=20; C=1; S=8000.0");
+  }
+
   @Test public void testSelectDistinct() {
     CalciteAssert.that()
         .with(CalciteAssert.Config.REGULAR)

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6d79b5eb/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java b/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
index 95e5279..afd3f2b 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
@@ -43,15 +43,19 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
     return DiffRepository.lookup(SqlToRelConverterTest.class);
   }
 
+  /** Sets the SQL statement for a test. */
+  public final Sql sql(String sql) {
+    return new Sql(sql);
+  }
+
   protected final void check(
       String sql,
       String plan) {
-    tester.assertConvertsTo(sql, plan);
+    sql(sql).convertsTo(plan);
   }
 
   @Test public void testIntegerLiteral() {
-    check(
-        "select 1 from emp", "${plan}");
+    check("select 1 from emp", "${plan}");
   }
 
   @Test public void testAliasList() {
@@ -193,9 +197,82 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
   }
 
   @Test public void testAggregateNoGroup() {
-    check(
-        "select sum(deptno) from emp",
-        "${plan}");
+    sql("select sum(deptno) from emp").ok();
+  }
+
+  @Test public void testGroupEmpty() {
+    sql("select sum(deptno) from emp group by ()").ok();
+  }
+
+  // Same effect as writing "GROUP BY deptno"
+  @Test public void testSingletonGroupingSet() {
+    sql("select sum(sal) from emp group by grouping sets (deptno)").ok();
+  }
+
+  @Test public void testGroupingSets() {
+    sql("select deptno, ename, sum(sal) from emp\n"
+        + "group by grouping sets ((deptno), (ename, deptno))\n"
+        + "order by 2").ok();
+  }
+
+  /**
+   * GROUP BY with duplicates
+   *
+   * <p>From SQL spec:
+   * <blockquote>NOTE 190 — That is, a simple <em>group by clause</em> that is
+   * not primitive may be transformed into a primitive <em>group by clause</em>
+   * by deleting all parentheses, and deleting extra commas as necessary for
+   * correct syntax. If there are no grouping columns at all (for example,
+   * GROUP BY (), ()), this is transformed to the canonical form GROUP BY ().
+   * </blockquote> */
+  // Same effect as writing "GROUP BY ()"
+  @Test public void testGroupByWithDuplicates() {
+    sql("select sum(sal) from emp group by (), ()").ok();
+  }
+
+  /** GROUP BY with duplicate (and heavily nested) GROUPING SETS. */
+  @Test public void testDuplicateGroupingSets() {
+    sql("select sum(sal) from emp\n"
+        + "group by sal,\n"
+        + "  grouping sets (deptno,\n"
+        + "    grouping sets ((deptno, ename), ename),\n"
+        + "      (ename)),\n"
+        + "  ()").ok();
+  }
+
+  @Test public void testGroupingSetsCartesianProduct() {
+    // Equivalent to (a, c), (a, d), (b, c), (b, d)
+    sql("select 1 from (values (1, 2, 3, 4)) as t(a, b, c, d)\n"
+        + "group by grouping sets (a, b), grouping sets (c, d)").ok();
+  }
+
+  @Test public void testGroupingSetsCartesianProduct2() {
+    sql("select 1 from (values (1, 2, 3, 4)) as t(a, b, c, d)\n"
+      + "group by grouping sets (a, (a, b)), grouping sets (c), d").ok();
+  }
+
+  @Test public void testRollup() {
+    // Equivalent to {(a, b), (a), ()}  * {(c, d), (c), ()}
+    sql("select 1 from (values (1, 2, 3, 4)) as t(a, b, c, d)\n"
+        + "group by rollup(a, b), rollup(c, d)").ok();
+  }
+
+  @Test public void testRollupTuples() {
+    // rollup(b, (a, d)) is (b, a, d), (b), ()
+    sql("select 1 from (values (1, 2, 3, 4)) as t(a, b, c, d)\n"
+        + "group by rollup(b, (a, d))").ok();
+  }
+
+  @Test public void testCube() {
+    // cube(a, b) is {(a, b), (a), (b), ()}
+    sql("select 1 from (values (1, 2, 3, 4)) as t(a, b, c, d)\n"
+        + "group by cube(a, b)").ok();
+  }
+
+  @Test public void testGroupingSetsWith() {
+    sql("with t(a, b, c, d) as (values (1, 2, 3, 4))\n"
+        + "select 1 from t\n"
+        + "group by rollup(a, b), rollup(c, d)").ok();
   }
 
   @Test public void testHaving() {
@@ -216,32 +293,24 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
 
   @Test public void testGroupBug281b() {
     // Try to confuse it with spurious columns.
-    check(
-        "select name, foo from ("
-            + "select deptno, name, count(deptno) as foo "
-            + "from dept "
-            + "group by name, deptno, name)",
-        "${plan}");
+    sql("select name, foo from ("
+        + "select deptno, name, count(deptno) as foo "
+        + "from dept "
+        + "group by name, deptno, name)").ok();
   }
 
   @Test public void testAggDistinct() {
-    check(
-        "select deptno, sum(sal), sum(distinct sal), count(*) "
-            + "from emp "
-            + "group by deptno",
-        "${plan}");
+    sql("select deptno, sum(sal), sum(distinct sal), count(*) "
+        + "from emp "
+        + "group by deptno").ok();
   }
 
   @Test public void testSelectDistinct() {
-    check(
-        "select distinct sal + 5 from emp",
-        "${plan}");
+    sql("select distinct sal + 5 from emp").ok();
   }
 
   @Test public void testSelectDistinctGroup() {
-    check(
-        "select distinct sum(sal) from emp group by deptno",
-        "${plan}");
+    sql("select distinct sum(sal) from emp group by deptno").ok();
   }
 
   /**
@@ -957,6 +1026,23 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
       super.visit(node, ordinal, parent);
     }
   }
+
+  /** Allows fluent testing. */
+  public class Sql {
+    private final String sql;
+
+    Sql(String sql) {
+      this.sql = sql;
+    }
+
+    public void ok() {
+      convertsTo("${plan}");
+    }
+
+    public void convertsTo(String plan) {
+      tester.assertConvertsTo(sql, plan);
+    }
+  }
 }
 
 // End SqlToRelConverterTest.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6d79b5eb/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
index 3f90177..ed15304 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
@@ -3775,33 +3775,16 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     checkExpType("ceil(interval '2' second)", "INTERVAL SECOND NOT NULL");
   }
 
-  protected void checkWin(String sql, String expectedMsgPattern) {
-    LOGGER.info(sql);
-    checkFails(sql, expectedMsgPattern);
-  }
-
-  public void checkWinClauseExp(String sql, String expectedMsgPattern) {
-    sql = "select * from emp " + sql;
-    checkWin(sql, expectedMsgPattern);
-  }
-
   public void checkWinFuncExpWithWinClause(
       String sql,
       String expectedMsgPattern) {
-    sql = "select " + sql + " from emp window w as (order by deptno)";
-    checkWin(sql, expectedMsgPattern);
-  }
-
-  public void checkWinFuncExp(String sql, String expectedMsgPattern) {
-    sql = "select " + sql + " from emp";
-    checkWin(sql, expectedMsgPattern);
+    winExp(sql).fails(expectedMsgPattern);
   }
 
   // test window partition clause. See SQL 2003 specification for detail
   public void _testWinPartClause() {
-    checkWinClauseExp(
-        "window w as (w2 order by deptno), w2 as (^rang^e 100 preceding)",
-        "Referenced window cannot have framing declarations");
+    win("window w as (w2 order by deptno), w2 as (^rang^e 100 preceding)")
+        .fails("Referenced window cannot have framing declarations");
     // Test specified collation, window clause syntax rule 4,5.
   }
 
@@ -3812,43 +3795,40 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     // <query specification> or <select statement: single row>,
     // or the <order by clause> of a simple table query.
     // See 4.15.3 for detail
-    checkWin(
-        "select *\n"
-            + " from emp\n"
-            + " where ^sum(sal) over (partition by deptno\n"
-            + "    order by empno\n"
-            + "    rows 3 preceding)^ > 10",
-        "Windowed aggregate expression is illegal in WHERE clause");
-
-    checkWin(
-        "select *\n"
-            + " from emp\n"
-            + " group by ename, ^sum(sal) over (partition by deptno\n"
-            + "    order by empno\n"
-            + "    rows 3 preceding)^ + 10\n"
-            + "order by deptno",
-        "Windowed aggregate expression is illegal in GROUP BY clause");
-
-    checkWin(
-        "select *\n"
-            + " from emp\n"
-            + " join dept on emp.deptno = dept.deptno\n"
-            + " and ^sum(sal) over (partition by deptno\n"
-            + "    order by empno\n"
-            + "    rows 3 preceding)^ = dept.deptno + 40\n"
-            + "order by deptno",
-        "Windowed aggregate expression is illegal in ON clause");
+    winSql("select *\n"
+        + " from emp\n"
+        + " where ^sum(sal) over (partition by deptno\n"
+        + "    order by empno\n"
+        + "    rows 3 preceding)^ > 10")
+        .fails("Windowed aggregate expression is illegal in WHERE clause");
+
+    winSql("select *\n"
+        + " from emp\n"
+        + " group by ename, ^sum(sal) over (partition by deptno\n"
+        + "    order by empno\n"
+        + "    rows 3 preceding)^ + 10\n"
+        + "order by deptno")
+        .fails("Windowed aggregate expression is illegal in GROUP BY clause");
+
+    winSql("select *\n"
+        + " from emp\n"
+        + " join dept on emp.deptno = dept.deptno\n"
+        + " and ^sum(sal) over (partition by deptno\n"
+        + "    order by empno\n"
+        + "    rows 3 preceding)^ = dept.deptno + 40\n"
+        + "order by deptno")
+        .fails("Windowed aggregate expression is illegal in ON clause");
 
     // rule 3, a)
-    checkWin(
-        "select sal from emp order by sum(sal) over (partition by deptno order by deptno)",
-        null);
+    winSql(
+        "select sal from emp order by sum(sal) over (partition by deptno order by deptno)")
+        .ok();
 
     // scope reference
 
     // rule 4,
     // valid window functions
-    checkWinFuncExpWithWinClause("sum(sal)", null);
+    winExp("sum(sal)").ok();
   }
 
   @Test public void testWindowFunctions2() {
@@ -3858,87 +3838,79 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
           "select rank() over (order by deptno) from emp",
           "INTEGER NOT NULL");
     }
-    checkWin(
-        "select rank() over w from emp\n"
-            + "window w as ^(partition by sal)^, w2 as (w order by deptno)",
-        "RANK or DENSE_RANK functions require ORDER BY clause in window specification");
-    checkWin(
-        "select rank() over w2 from emp\n"
-            + "window w as (partition by sal), w2 as (w order by deptno)",
-        null);
+    winSql("select rank() over w from emp\n"
+        + "window w as ^(partition by sal)^, w2 as (w order by deptno)")
+        .fails(
+            "RANK or DENSE_RANK functions require ORDER BY clause in window specification");
+    winSql("select rank() over w2 from emp\n"
+        + "window w as (partition by sal), w2 as (w order by deptno)").ok();
     // row_number function
-    checkWinFuncExpWithWinClause(
-        "row_number() over (order by deptno)",
-        null);
+    winExp("row_number() over (order by deptno)").ok();
 
     // rank function type
     if (defined.contains("DENSE_RANK")) {
-      checkWinFuncExpWithWinClause("dense_rank()", null);
+      winExp("dense_rank()").ok();
     } else {
       checkWinFuncExpWithWinClause(
           "^dense_rank()^",
           "Function 'DENSE_RANK\\(\\)' is not defined");
     }
-    checkWinFuncExpWithWinClause("rank() over (order by empno)", null);
-    checkWinFuncExpWithWinClause(
-        "percent_rank() over (order by empno)",
-        null);
-    checkWinFuncExpWithWinClause(
-        "cume_dist() over (order by empno)",
-        null);
+    winExp("rank() over (order by empno)").ok();
+    winExp("percent_rank() over (order by empno)").ok();
+    winExp("cume_dist() over (order by empno)").ok();
 
     // rule 6a
     // ORDER BY required with RANK & DENSE_RANK
-    checkWin(
-        "select rank() over ^(partition by deptno)^ from emp",
-        "RANK or DENSE_RANK functions require ORDER BY clause in window specification");
-    checkWin(
-        "select dense_rank() over ^(partition by deptno)^ from emp ",
-        "RANK or DENSE_RANK functions require ORDER BY clause in window specification");
-    checkWin(
-        "select rank() over w from emp window w as ^(partition by deptno)^",
-        "RANK or DENSE_RANK functions require ORDER BY clause in window specification");
-    checkWin(
-        "select dense_rank() over w from emp window w as ^(partition by deptno)^",
-        "RANK or DENSE_RANK functions require ORDER BY clause in window specification");
+    winSql("select rank() over ^(partition by deptno)^ from emp")
+        .fails(
+            "RANK or DENSE_RANK functions require ORDER BY clause in window specification");
+    winSql("select dense_rank() over ^(partition by deptno)^ from emp ")
+        .fails(
+            "RANK or DENSE_RANK functions require ORDER BY clause in window specification");
+    winSql("select rank() over w from emp window w as ^(partition by deptno)^")
+        .fails(
+            "RANK or DENSE_RANK functions require ORDER BY clause in window specification");
+    winSql(
+        "select dense_rank() over w from emp window w as ^(partition by deptno)^")
+        .fails(
+            "RANK or DENSE_RANK functions require ORDER BY clause in window specification");
 
     // rule 6b
     // Framing not allowed with RANK & DENSE_RANK functions
     // window framing defined in window clause
-    checkWin(
-        "select rank() over w from emp window w as (order by empno ^rows^ 2 preceding )",
-        "ROW/RANGE not allowed with RANK or DENSE_RANK functions");
-    checkWin(
-        "select dense_rank() over w from emp window w as (order by empno ^rows^ 2 preceding)",
-        "ROW/RANGE not allowed with RANK or DENSE_RANK functions");
+    winSql(
+        "select rank() over w from emp window w as (order by empno ^rows^ 2 preceding )")
+        .fails("ROW/RANGE not allowed with RANK or DENSE_RANK functions");
+    winSql(
+        "select dense_rank() over w from emp window w as (order by empno ^rows^ 2 preceding)")
+        .fails("ROW/RANGE not allowed with RANK or DENSE_RANK functions");
     if (defined.contains("PERCENT_RANK")) {
-      checkWin(
-          "select percent_rank() over w from emp window w as (rows 2 preceding )",
-          null);
+      winSql(
+          "select percent_rank() over w from emp window w as (rows 2 preceding )")
+          .ok();
     } else {
       checkWinFuncExpWithWinClause(
           "^percent_rank()^",
           "Function 'PERCENT_RANK\\(\\)' is not defined");
     }
     if (defined.contains("CUME_DIST")) {
-      checkWin(
-          "select cume_dist() over w from emp window w as (rows 2 preceding)",
-          null);
-      checkWin("select cume_dist() over (rows 2 preceding ) from emp ", null);
+      winSql(
+          "select cume_dist() over w from emp window w as (rows 2 preceding)")
+          .ok();
+      winSql("select cume_dist() over (rows 2 preceding ) from emp ").ok();
     } else {
       checkWinFuncExpWithWinClause(
           "^cume_dist()^",
           "Function 'CUME_DIST\\(\\)' is not defined");
     }
     // window framing defined in in-line window
-    checkWin(
-        "select rank() over (order by empno ^range^ 2 preceding ) from emp ",
-        "ROW/RANGE not allowed with RANK or DENSE_RANK functions");
-    checkWin(
-        "select dense_rank() over (order by empno ^rows^ 2 preceding ) from emp ",
-        "ROW/RANGE not allowed with RANK or DENSE_RANK functions");
+    winSql("select rank() over (order by empno ^range^ 2 preceding ) from emp ")
+        .fails("ROW/RANGE not allowed with RANK or DENSE_RANK functions");
+    winSql(
+        "select dense_rank() over (order by empno ^rows^ 2 preceding ) from emp ")
+        .fails("ROW/RANGE not allowed with RANK or DENSE_RANK functions");
     if (defined.contains("PERCENT_RANK")) {
-      checkWin("select percent_rank() over (rows 2 preceding ) from emp", null);
+      winSql("select percent_rank() over (rows 2 preceding ) from emp").ok();
     }
 
     // invalid column reference
@@ -3956,31 +3928,31 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     // checkWinFuncExpWithWinClause(" sum(distinct sal) over w ", null);
 
     // 7.11 rule 10c
-    checkWin(
-        "select sum(sal) over (w partition by ^deptno^)\n"
-            + " from emp window w as (order by empno rows 2 preceding )",
-        "PARTITION BY not allowed with existing window reference");
+    winSql("select sum(sal) over (w partition by ^deptno^)\n"
+        + " from emp window w as (order by empno rows 2 preceding )")
+        .fails("PARTITION BY not allowed with existing window reference");
 
     // 7.11 rule 10d
-    checkWin(
-        "select sum(sal) over (w order by ^empno^)\n"
-            + " from emp window w as (order by empno rows 2 preceding )",
-        "ORDER BY not allowed in both base and referenced windows");
+    winSql("select sum(sal) over (w order by ^empno^)\n"
+        + " from emp window w as (order by empno rows 2 preceding )")
+        .fails("ORDER BY not allowed in both base and referenced windows");
 
     // 7.11 rule 10e
-    checkWin("select sum(sal) over (w)\n"
-            + " from emp window w as (order by empno ^rows^ 2 preceding )",
-        "Referenced window cannot have framing declarations");
+    winSql("select sum(sal) over (w)\n"
+        + " from emp window w as (order by empno ^rows^ 2 preceding )")
+        .fails("Referenced window cannot have framing declarations");
 
     // Empty window is OK for functions that don't require ordering.
-    checkWin("select sum(sal) over () from emp", null);
-    checkWin("select sum(sal) over w from emp window w as ()", null);
-    checkWin("select count(*) over () from emp", null);
-    checkWin("select count(*) over w from emp window w as ()", null);
-    checkWin("select rank() over ^()^ from emp",
-        "RANK or DENSE_RANK functions require ORDER BY clause in window specification");
-    checkWin("select rank() over w from emp window w as ^()^",
-        "RANK or DENSE_RANK functions require ORDER BY clause in window specification");
+    winSql("select sum(sal) over () from emp").ok();
+    winSql("select sum(sal) over w from emp window w as ()").ok();
+    winSql("select count(*) over () from emp").ok();
+    winSql("select count(*) over w from emp window w as ()").ok();
+    winSql("select rank() over ^()^ from emp")
+        .fails(
+            "RANK or DENSE_RANK functions require ORDER BY clause in window specification");
+    winSql("select rank() over w from emp window w as ^()^")
+        .fails(
+            "RANK or DENSE_RANK functions require ORDER BY clause in window specification");
   }
 
   @Test public void testInlineWinDef() {
@@ -3988,136 +3960,102 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     // fully defined in SQL 03 Std. section 7.1 <window clause>
     check("select sum(sal) over (partition by deptno order by empno)\n"
         + "from emp order by empno");
-    checkWinFuncExp(
-        "sum(sal) OVER ("
-            + "partition by deptno "
-            + "order by empno "
-            + "rows 2 preceding )",
-        null);
-    checkWinFuncExp(
-        "sum(sal) OVER ("
-            + "order by 1 "
-            + "rows 2 preceding )",
-        null);
-    checkWinFuncExp(
-        "sum(sal) OVER ("
-            + "order by 'b' "
-            + "rows 2 preceding )",
-        null);
-    checkWinFuncExp(
-        "sum(sal) over ("
-            + "partition by deptno "
-            + "order by 1+1 rows 26 preceding)",
-        null);
-    checkWinFuncExp(
-        "sum(sal) over (order by deptno rows unbounded preceding)",
-        null);
-    checkWinFuncExp(
-        "sum(sal) over (order by deptno rows current row)",
-        null);
-    checkWinFuncExp(
-        "sum(sal) over ^("
-            + "order by deptno "
-            + "rows between unbounded preceding and unbounded following)^",
-        null);
-    checkWinFuncExp(
-        "sum(sal) over ^("
-            + "order by deptno "
-            + "rows between CURRENT ROW and unbounded following)^",
-        null);
-    checkWinFuncExp(
-        "sum(sal) over ("
-            + "order by deptno "
-            + "rows between unbounded preceding and CURRENT ROW)",
-        null);
+    winExp2("sum(sal) OVER ("
+        + "partition by deptno "
+        + "order by empno "
+        + "rows 2 preceding )").ok();
+    winExp2("sum(sal) OVER ("
+        + "order by 1 "
+        + "rows 2 preceding )").ok();
+    winExp2("sum(sal) OVER ("
+        + "order by 'b' "
+        + "rows 2 preceding )").ok();
+    winExp2("sum(sal) over ("
+        + "partition by deptno "
+        + "order by 1+1 rows 26 preceding)").ok();
+    winExp2("sum(sal) over (order by deptno rows unbounded preceding)").ok();
+    winExp2("sum(sal) over (order by deptno rows current row)").ok();
+    winExp2("sum(sal) over ^("
+        + "order by deptno "
+        + "rows between unbounded preceding and unbounded following)^").ok();
+    winExp2("sum(sal) over ^("
+        + "order by deptno "
+        + "rows between CURRENT ROW and unbounded following)^").ok();
+    winExp2("sum(sal) over ("
+        + "order by deptno "
+        + "rows between unbounded preceding and CURRENT ROW)").ok();
 
     // logical current row/current row
-    checkWinFuncExp(
-        "sum(sal) over ("
-            + "order by deptno "
-            + "rows between CURRENT ROW and CURRENT ROW)",
-        null);
+    winExp2("sum(sal) over ("
+        + "order by deptno "
+        + "rows between CURRENT ROW and CURRENT ROW)").ok();
 
     // physical current row/current row
-    checkWinFuncExp(
-        "sum(sal) over ("
-            + "order by deptno "
-            + "range between CURRENT ROW and CURRENT ROW)",
-        null);
+    winExp2("sum(sal) over ("
+        + "order by deptno "
+        + "range between CURRENT ROW and CURRENT ROW)").ok();
 
-    checkWinFuncExp(
-        "sum(sal) over ("
-            + "order by deptno "
-            + "rows between 2 preceding and CURRENT ROW)",
-        null);
-    checkWinFuncExpWithWinClause(
-        "sum(sal) OVER (w "
-            + "rows 2 preceding )",
-        null);
-    checkWinFuncExp(
-        "sum(sal) over (order by deptno range 2.0 preceding)",
-        null);
+    winExp2("sum(sal) over ("
+        + "order by deptno "
+        + "rows between 2 preceding and CURRENT ROW)").ok();
+    winExp("sum(sal) OVER (w "
+        + "rows 2 preceding )").ok();
+    winExp2("sum(sal) over (order by deptno range 2.0 preceding)").ok();
 
     // Failure mode tests
-    checkWinFuncExp(
-        "sum(sal) over (order by deptno "
-            + "rows between ^UNBOUNDED FOLLOWING^ and unbounded preceding)",
-        "UNBOUNDED FOLLOWING cannot be specified for the lower frame boundary");
-    checkWinFuncExp(
-        "sum(sal) over ("
-            + "order by deptno "
-            + "rows between 2 preceding and ^UNBOUNDED PRECEDING^)",
-        "UNBOUNDED PRECEDING cannot be specified for the upper frame boundary");
-    checkWinFuncExp(
-        "sum(sal) over ("
-            + "order by deptno "
-            + "rows between CURRENT ROW and ^2 preceding^)",
-        "Upper frame boundary cannot be PRECEDING when lower boundary is CURRENT ROW");
-    checkWinFuncExp(
-        "sum(sal) over ("
-            + "order by deptno "
-            + "rows between 2 following and ^CURRENT ROW^)",
-        "Upper frame boundary cannot be CURRENT ROW when lower boundary is FOLLOWING");
-    checkWinFuncExp(
-        "sum(sal) over ("
-            + "order by deptno "
-            + "rows between 2 following and ^2 preceding^)",
-        "Upper frame boundary cannot be PRECEDING when lower boundary is FOLLOWING");
-    checkWinFuncExp(
-        "sum(sal) over ("
-            + "order by deptno "
-            + "RANGE BETWEEN ^INTERVAL '1' SECOND^ PRECEDING AND INTERVAL '1' SECOND FOLLOWING)",
-        "Data Type mismatch between ORDER BY and RANGE clause");
-    checkWinFuncExp(
-        "sum(sal) over ("
-            + "order by empno "
-            + "RANGE BETWEEN ^INTERVAL '1' SECOND^ PRECEDING AND INTERVAL '1' SECOND FOLLOWING)",
-        "Data Type mismatch between ORDER BY and RANGE clause");
-    checkWinFuncExp(
-        "sum(sal) over (order by deptno, empno ^range^ 2 preceding)",
-        "RANGE clause cannot be used with compound ORDER BY clause");
-    checkWinFuncExp(
-        "sum(sal) over ^(partition by deptno range 5 preceding)^",
-        "Window specification must contain an ORDER BY clause");
-    checkWinFuncExp(
-        "sum(sal) over ^w1^",
-        "Window 'W1' not found");
-    checkWinFuncExp(
-        "sum(sal) OVER (^w1^ "
-            + "partition by deptno "
-            + "order by empno "
-            + "rows 2 preceding )",
-        "Window 'W1' not found");
+    winExp2("sum(sal) over (order by deptno "
+        + "rows between ^UNBOUNDED FOLLOWING^ and unbounded preceding)")
+        .fails(
+            "UNBOUNDED FOLLOWING cannot be specified for the lower frame boundary");
+    winExp2("sum(sal) over ("
+        + "order by deptno "
+        + "rows between 2 preceding and ^UNBOUNDED PRECEDING^)")
+        .fails(
+            "UNBOUNDED PRECEDING cannot be specified for the upper frame boundary");
+    winExp2("sum(sal) over ("
+        + "order by deptno "
+        + "rows between CURRENT ROW and ^2 preceding^)")
+        .fails(
+            "Upper frame boundary cannot be PRECEDING when lower boundary is CURRENT ROW");
+    winExp2("sum(sal) over ("
+        + "order by deptno "
+        + "rows between 2 following and ^CURRENT ROW^)")
+        .fails(
+            "Upper frame boundary cannot be CURRENT ROW when lower boundary is FOLLOWING");
+    winExp2("sum(sal) over ("
+        + "order by deptno "
+        + "rows between 2 following and ^2 preceding^)")
+        .fails(
+            "Upper frame boundary cannot be PRECEDING when lower boundary is FOLLOWING");
+    winExp2("sum(sal) over ("
+        + "order by deptno "
+        + "RANGE BETWEEN ^INTERVAL '1' SECOND^ PRECEDING AND INTERVAL '1' SECOND FOLLOWING)")
+        .fails("Data Type mismatch between ORDER BY and RANGE clause");
+    winExp2("sum(sal) over ("
+        + "order by empno "
+        + "RANGE BETWEEN ^INTERVAL '1' SECOND^ PRECEDING AND INTERVAL '1' SECOND FOLLOWING)")
+        .fails("Data Type mismatch between ORDER BY and RANGE clause");
+    winExp2("sum(sal) over (order by deptno, empno ^range^ 2 preceding)")
+        .fails("RANGE clause cannot be used with compound ORDER BY clause");
+    winExp2("sum(sal) over ^(partition by deptno range 5 preceding)^")
+        .fails("Window specification must contain an ORDER BY clause");
+    winExp2("sum(sal) over ^w1^").fails("Window 'W1' not found");
+    winExp2("sum(sal) OVER (^w1^ "
+        + "partition by deptno "
+        + "order by empno "
+        + "rows 2 preceding )")
+        .fails("Window 'W1' not found");
   }
 
   @Test public void testPartitionByExpr() {
-    checkWinFuncExp(
-        "sum(sal) over (partition by empno + deptno order by empno range 5 preceding)",
-        null);
+    winExp2(
+        "sum(sal) over (partition by empno + deptno order by empno range 5 preceding)")
+        .ok();
 
-    checkWinFuncExp(
-        "sum(sal) over (partition by ^empno + ename^ order by empno range 5 preceding)",
-        "(?s)Cannot apply '\\+' to arguments of type '<INTEGER> \\+ <VARCHAR\\(20\\)>'.*");
+    winExp2(
+        "sum(sal) over (partition by ^empno + ename^ order by empno range 5 preceding)")
+        .fails(
+            "(?s)Cannot apply '\\+' to arguments of type '<INTEGER> \\+ <VARCHAR\\(20\\)>'.*");
   }
 
   @Test public void testWindowClause() {
@@ -4125,13 +4063,12 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     // --   positive testings           --
     // -----------------------------------
     // correct syntax:
-    checkWinFuncExpWithWinClause("sum(sal) as sumsal", null);
-    checkWinClauseExp(
-        "window w as (partition by sal order by deptno rows 2 preceding)",
-        null);
+    winExp("sum(sal) as sumsal").ok();
+    win("window w as (partition by sal order by deptno rows 2 preceding)")
+        .ok();
 
     // define window on an existing window
-    checkWinClauseExp("window w as (order by sal), w1 as (w)", null);
+    win("window w as (order by sal), w1 as (w)").ok();
 
     // -----------------------------------
     // --   negative testings           --
@@ -4143,117 +4080,99 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     // rule 11
     // a)
     // missing window order clause.
-    checkWinClauseExp(
-        "window w as ^(range 100 preceding)^",
-        "Window specification must contain an ORDER BY clause");
+    win("window w as ^(range 100 preceding)^")
+        .fails("Window specification must contain an ORDER BY clause");
 
     // order by number
-    checkWinClauseExp(
-        "window w as (order by sal range 100 preceding)",
-        null);
+    win("window w as (order by sal range 100 preceding)").ok();
 
     // order by date
-    checkWinClauseExp(
-        "window w as (order by hiredate range ^100^ preceding)",
-        "Data Type mismatch between ORDER BY and RANGE clause");
+    win("window w as (order by hiredate range ^100^ preceding)")
+        .fails("Data Type mismatch between ORDER BY and RANGE clause");
 
     // order by string, should fail
-    checkWinClauseExp(
-        "window w as (order by ename range ^100^ preceding)",
-        "Data type of ORDER BY prohibits use of RANGE clause");
+    win("window w as (order by ename range ^100^ preceding)")
+        .fails("Data type of ORDER BY prohibits use of RANGE clause");
     // todo: interval test ???
 
     // b)
     // valid
-    checkWinClauseExp("window w as (rows 2 preceding)", null);
+    win("window w as (rows 2 preceding)").ok();
 
     // invalid tests exact numeric for the unsigned value specification The
     // following two test fail as they should but in the parser: JR not
     // anymore now the validator kicks out
-    checkWinClauseExp(
-        "window w as (rows ^-2.5^ preceding)",
-        "ROWS value must be a non-negative integral constant");
-    checkWinClauseExp(
-        "window w as (rows ^-2^ preceding)",
-        "ROWS value must be a non-negative integral constant");
+    win("window w as (rows ^-2.5^ preceding)")
+        .fails("ROWS value must be a non-negative integral constant");
+    win("window w as (rows ^-2^ preceding)")
+        .fails("ROWS value must be a non-negative integral constant");
 
     // This test should fail as per 03 Std. but we pass it and plan
     // to apply the FLOOR function before window processing
-    checkWinClauseExp(
-        "window w as (rows ^2.5^ preceding)",
-        "ROWS value must be a non-negative integral constant");
+    win("window w as (rows ^2.5^ preceding)")
+        .fails("ROWS value must be a non-negative integral constant");
 
     // -----------------------------------
     // --   negative testings           --
     // -----------------------------------
     // reference undefined xyz column
-    checkWinClauseExp(
-        "window w as (partition by ^xyz^)",
-        "Column 'XYZ' not found in any table");
+    win("window w as (partition by ^xyz^)")
+        .fails("Column 'XYZ' not found in any table");
 
     // window definition is empty when applied to unsorted table
-    checkWinClauseExp(
-        "window w as ^( /* boo! */  )^",
-        null);
+    win("window w as ^( /* boo! */  )^").ok();
 
     // duplicate window name
-    checkWinClauseExp(
-        "window w as (order by empno), ^w^ as (order by empno)",
-        "Duplicate window names not allowed");
-    checkWinClauseExp(
-        "window win1 as (order by empno), ^win1^ as (order by empno)",
-        "Duplicate window names not allowed");
+    win("window w as (order by empno), ^w^ as (order by empno)")
+        .fails("Duplicate window names not allowed");
+    win("window win1 as (order by empno), ^win1^ as (order by empno)")
+        .fails("Duplicate window names not allowed");
 
     // syntax rule 6
-    checkFails(
-        "select min(sal) over (order by deptno) from emp group by deptno,sal",
-        null);
+    sql("select min(sal) over (order by deptno) from emp group by deptno,sal")
+        .ok();
     checkFails(
         "select min(sal) over (order by ^deptno^) from emp group by sal",
         "Expression 'DEPTNO' is not being grouped");
-    checkFails(
-        "select min(sal) over\n"
-            + "(partition by comm order by deptno) from emp group by deptno,sal,comm",
-        null);
-    checkFails(
-        "select min(sal) over\n"
-            + "(partition by ^comm^ order by deptno) from emp group by deptno,sal",
+    sql("select min(sal) over\n"
+        + "(partition by comm order by deptno) from emp group by deptno,sal,comm")
+        .ok();
+    checkFails("select min(sal) over\n"
+        + "(partition by ^comm^ order by deptno) from emp group by deptno,sal",
         "Expression 'COMM' is not being grouped");
 
     // syntax rule 7
-    checkWinClauseExp(
-        "window w as (order by rank() over (order by sal))",
-        null);
+    win("window w as (order by rank() over (order by sal))").ok();
 
     // ------------------------------------
     // ---- window frame between tests ----
     // ------------------------------------
     // bound 1 shall not specify UNBOUNDED FOLLOWING
-    checkWinClauseExp(
-        "window w as (rows between ^unbounded following^ and 5 following)",
-        "UNBOUNDED FOLLOWING cannot be specified for the lower frame boundary");
+    win("window w as (rows between ^unbounded following^ and 5 following)")
+        .fails(
+            "UNBOUNDED FOLLOWING cannot be specified for the lower frame boundary");
 
     // bound 2 shall not specify UNBOUNDED PRECEDING
-    checkWinClauseExp(
-        "window w as ("
-            + "order by deptno "
-            + "rows between 2 preceding and ^UNBOUNDED PRECEDING^)",
-        "UNBOUNDED PRECEDING cannot be specified for the upper frame boundary");
-    checkWinClauseExp(
-        "window w as ("
-            + "order by deptno "
-            + "rows between 2 following and ^2 preceding^)",
-        "Upper frame boundary cannot be PRECEDING when lower boundary is FOLLOWING");
-    checkWinClauseExp(
-        "window w as ("
-            + "order by deptno "
-            + "rows between CURRENT ROW and ^2 preceding^)",
-        "Upper frame boundary cannot be PRECEDING when lower boundary is CURRENT ROW");
-    checkWinClauseExp(
-        "window w as ("
-            + "order by deptno "
-            + "rows between 2 following and ^CURRENT ROW^)",
-        "Upper frame boundary cannot be CURRENT ROW when lower boundary is FOLLOWING");
+    win("window w as ("
+        + "order by deptno "
+        + "rows between 2 preceding and ^UNBOUNDED PRECEDING^)")
+        .fails(
+            "UNBOUNDED PRECEDING cannot be specified for the upper frame boundary");
+    win("window w as ("
+        + "order by deptno "
+        + "rows between 2 following and ^2 preceding^)")
+        .fails(
+            "Upper frame boundary cannot be PRECEDING when lower boundary is FOLLOWING");
+    win("window w as ("
+        + "order by deptno "
+        + "rows between CURRENT ROW and ^2 preceding^)")
+        .fails(
+            "Upper frame boundary cannot be PRECEDING when lower boundary is CURRENT ROW");
+    win("window w as ("
+        + "order by deptno "
+        + "rows between 2 following and ^CURRENT ROW^)")
+        .fails(
+            "Upper frame boundary cannot be CURRENT ROW when lower boundary is FOLLOWING");
 
     // Sql '03 rule 10 c) assertExceptionIsThrown("select deptno as d, sal
     // as s from emp window w as (partition by deptno order by sal), w2 as
@@ -4261,46 +4180,38 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     // (partition by sal order by deptno), w2 as (w partition by sal)",
     // null); d) valid because existing window does not have an ORDER BY
     // clause
-    checkWinClauseExp(
-        "window w as (w2 range 2 preceding ), w2 as (order by sal)",
-        null);
-    checkWinClauseExp(
-        "window w as ^(partition by sal)^, w2 as (w order by deptno)",
-        null);
-    checkWinClauseExp(
-        "window w as (w2 partition by ^sal^), w2 as (order by deptno)",
-        "PARTITION BY not allowed with existing window reference");
-    checkWinClauseExp(
-        "window w as (partition by sal order by deptno), w2 as (w order by ^deptno^)",
-        "ORDER BY not allowed in both base and referenced windows");
+    win("window w as (w2 range 2 preceding ), w2 as (order by sal)").ok();
+    win("window w as ^(partition by sal)^, w2 as (w order by deptno)").ok();
+    win("window w as (w2 partition by ^sal^), w2 as (order by deptno)")
+        .fails("PARTITION BY not allowed with existing window reference");
+    win(
+        "window w as (partition by sal order by deptno), w2 as (w order by ^deptno^)")
+        .fails("ORDER BY not allowed in both base and referenced windows");
 
     // e)
-    checkWinClauseExp(
-        "window w as (w2 order by deptno), w2 as (^range^ 100 preceding)",
-        "Referenced window cannot have framing declarations");
+    win("window w as (w2 order by deptno), w2 as (^range^ 100 preceding)")
+        .fails("Referenced window cannot have framing declarations");
 
     // rule 12, todo: test scope of window assertExceptionIsThrown("select
     // deptno as d from emp window d as (partition by deptno)", null);
 
     // rule 13
-    checkWinClauseExp("window w as (order by sal)", null);
-    checkWinClauseExp(
-        "window w as (order by ^non_exist_col^)",
-        "Column 'NON_EXIST_COL' not found in any table");
-    checkWinClauseExp(
-        "window w as (partition by ^non_exist_col^ order by sal)",
-        "Column 'NON_EXIST_COL' not found in any table");
+    win("window w as (order by sal)").ok();
+    win("window w as (order by ^non_exist_col^)")
+        .fails("Column 'NON_EXIST_COL' not found in any table");
+    win("window w as (partition by ^non_exist_col^ order by sal)")
+        .fails("Column 'NON_EXIST_COL' not found in any table");
   }
 
   @Test public void testWindowClause2() {
     // 7.10 syntax rule 2 <new window name> NWN1 shall not be contained in
     // the scope of another <new window name> NWN2 such that NWN1 and NWN2
     // are equivalent.
-    checkWinClauseExp(
-        "window\n"
-            + "w  as (partition by deptno order by empno rows 2 preceding),\n"
-            + "w2 as ^(partition by deptno order by empno rows 2 preceding)^\n",
-        "Duplicate window specification not allowed in the same window clause");
+    win("window\n"
+        + "w  as (partition by deptno order by empno rows 2 preceding),\n"
+        + "w2 as ^(partition by deptno order by empno rows 2 preceding)^\n")
+        .fails(
+            "Duplicate window specification not allowed in the same window clause");
   }
 
   @Test public void testWindowClauseWithSubquery() {
@@ -4372,9 +4283,8 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
   }
 
   @Test public void testOneWinFunc() {
-    checkWinClauseExp(
-        "window w as (partition by sal order by deptno rows 2 preceding)",
-        null);
+    win("window w as (partition by sal order by deptno rows 2 preceding)")
+        .ok();
   }
 
   @Test public void testNameResolutionInValuesClause() {
@@ -5449,12 +5359,11 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
         + "group by empno, deptno "
         + "order by x * sum(sal + 2)");
 
-    checkFails(
-        "select empno as x "
-            + "from emp "
-            + "group by empno, deptno "
-            + "order by empno * sum(sal + 2)",
-        tester.getConformance().isSortByAliasObscures() ? "xxxx" : null);
+    sql("select empno as x "
+        + "from emp "
+        + "group by empno, deptno "
+        + "order by empno * sum(sal + 2)")
+        .failsIf(tester.getConformance().isSortByAliasObscures(), "xxxx");
 
     // Distinct on expressions with attempts to order on a column in
     // the underlying table
@@ -5478,21 +5387,18 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     // These tests are primarily intended to test cases where sorting by
     // an alias is allowed.  But for instances that don't support sorting
     // by alias, the tests also verify that a proper exception is thrown.
-    checkFails(
-        "select distinct cast(empno as bigint) as empno "
-            + "from emp order by ^empno^",
-        tester.getConformance().isSortByAlias() ? null
-            : "Expression 'EMPNO' is not in the select clause");
-    checkFails(
-        "select distinct cast(empno as bigint) as eno "
-            + "from emp order by ^eno^",
-        tester.getConformance().isSortByAlias() ? null
-            : "Column 'ENO' not found in any table");
-    checkFails(
-        "select distinct cast(empno as bigint) as empno "
-            + "from emp e order by ^empno^",
-        tester.getConformance().isSortByAlias() ? null
-            : "Expression 'EMPNO' is not in the select clause");
+    sql("select distinct cast(empno as bigint) as empno "
+        + "from emp order by ^empno^")
+        .failsIf(!tester.getConformance().isSortByAlias(),
+            "Expression 'EMPNO' is not in the select clause");
+    sql("select distinct cast(empno as bigint) as eno "
+        + "from emp order by ^eno^")
+        .failsIf(!tester.getConformance().isSortByAlias(),
+            "Column 'ENO' not found in any table");
+    sql("select distinct cast(empno as bigint) as empno "
+        + "from emp e order by ^empno^")
+        .failsIf(!tester.getConformance().isSortByAlias(),
+            "Expression 'EMPNO' is not in the select clause");
 
     // Distinct on expressions, sorting using ordinals.
     if (tester.getConformance().isSortByOrdinal()) {
@@ -5560,6 +5466,28 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     check("select localtime, deptno + 3 from emp group by deptno");
   }
 
+  @Test public void testGroupingSets() {
+    sql("select count(1), ^empno^ from emp group by grouping sets (deptno)")
+        .fails("Expression 'EMPNO' is not being grouped");
+    sql("select deptno, ename, sum(sal) from emp\n"
+        + "group by grouping sets ((deptno), (ename, deptno))\n"
+        + "order by 2").ok();
+
+    // duplicate (and heavily nested) GROUPING SETS
+    sql("select sum(sal) from emp\n"
+        + "group by deptno,\n"
+        + "  grouping sets (deptno,\n"
+        + "    grouping sets (deptno, ename),\n"
+        + "      (ename)),\n"
+        + "  ()").ok();
+  }
+
+  @Test public void testRollup() {
+    sql("select deptno, count(*) as c, sum(sal) as s\n"
+        + "from emp\n"
+        + "group by rollup(deptno)").ok();
+  }
+
   @Test public void testGroupByCorrelatedColumnFails() {
     // -- this is not sql 2003 standard
     // -- see sql2003 part2,  7.9

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6d79b5eb/core/src/test/java/org/apache/calcite/test/SqlValidatorTestCase.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/SqlValidatorTestCase.java b/core/src/test/java/org/apache/calcite/test/SqlValidatorTestCase.java
index 42b9b11..dddffc9 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlValidatorTestCase.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlValidatorTestCase.java
@@ -81,8 +81,28 @@ public class SqlValidatorTestCase {
     return new SqlTesterImpl(DefaultSqlTestFactory.INSTANCE);
   }
 
+  public final Sql sql(String sql) {
+    return new Sql(sql);
+  }
+
+  public final Sql winSql(String sql) {
+    return sql(sql);
+  }
+
+  public final Sql win(String sql) {
+    return sql("select * from emp " + sql);
+  }
+
+  public Sql winExp(String sql) {
+    return winSql("select " + sql + " from emp window w as (order by deptno)");
+  }
+
+  public Sql winExp2(String sql) {
+    return winSql("select " + sql + " from emp");
+  }
+
   public void check(String sql) {
-    tester.assertExceptionIsThrown(sql, null);
+    sql(sql).ok();
   }
 
   public void checkExp(String sql) {
@@ -98,7 +118,7 @@ public class SqlValidatorTestCase {
   public final void checkFails(
       String sql,
       String expected) {
-    tester.assertExceptionIsThrown(sql, expected);
+    sql(sql).fails(expected);
   }
 
   /**
@@ -319,12 +339,15 @@ public class SqlValidatorTestCase {
             || (actualEndColumn <= 0)
             || (actualEndLine <= 0)) {
           if (sap.pos != null) {
-            throw new AssertionError("Expected error to have position,"
-                + " but actual error did not: "
-                + " actual pos [line " + actualLine
-                + " col " + actualColumn
-                + " thru line " + actualEndLine
-                + " col " + actualEndColumn + "]");
+            AssertionError e =
+                new AssertionError("Expected error to have position,"
+                    + " but actual error did not: "
+                    + " actual pos [line " + actualLine
+                    + " col " + actualColumn
+                    + " thru line " + actualEndLine + " col "
+                    + actualEndColumn + "]");
+            e.initCause(actualException);
+            throw e;
           }
           sqlWithCarets = sap.sql;
         } else {
@@ -510,6 +533,31 @@ public class SqlValidatorTestCase {
 
     SqlConformance getConformance();
   }
+
+  /** Fluent testing API. */
+  class Sql {
+    private final String sql;
+
+    Sql(String sql) {
+      this.sql = sql;
+    }
+
+    void ok() {
+      tester.assertExceptionIsThrown(sql, null);
+    }
+
+    void fails(String expected) {
+      tester.assertExceptionIsThrown(sql, expected);
+    }
+
+    void failsIf(boolean b, String expected) {
+      if (b) {
+        fails(expected);
+      } else {
+        ok();
+      }
+    }
+  }
 }
 
 // End SqlValidatorTestCase.java


[22/58] [abbrv] [partial] incubator-calcite git commit: [CALCITE-306] Standardize code style for "import package.*; "

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql2rel/RelDecorrelator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql2rel/RelDecorrelator.java b/core/src/main/java/org/apache/calcite/sql2rel/RelDecorrelator.java
index 6965175..4da6445 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/RelDecorrelator.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/RelDecorrelator.java
@@ -14,33 +14,92 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql2rel;
-
-import java.math.*;
-import java.util.*;
-import java.util.logging.*;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.rel.Correlation;
-import org.eigenbase.rel.metadata.*;
-import org.eigenbase.rel.rules.PushFilterPastJoinRule;
-import org.eigenbase.relopt.*;
-import org.eigenbase.relopt.hep.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.rex.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.fun.*;
-import org.eigenbase.trace.*;
-import org.eigenbase.util.*;
-import org.eigenbase.util.mapping.Mappings;
-
-import net.hydromatic.linq4j.Ord;
-import net.hydromatic.linq4j.function.Function2;
-
-import net.hydromatic.optiq.util.BitSets;
+package org.apache.calcite.sql2rel;
+
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.linq4j.function.Function2;
+import org.apache.calcite.plan.Context;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCostImpl;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.hep.HepPlanner;
+import org.apache.calcite.plan.hep.HepProgram;
+import org.apache.calcite.plan.hep.HepRelVertex;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelShuttleImpl;
+import org.apache.calcite.rel.RelVisitor;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.core.Correlation;
+import org.apache.calcite.rel.core.Correlator;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.logical.LogicalAggregate;
+import org.apache.calcite.rel.logical.LogicalFilter;
+import org.apache.calcite.rel.logical.LogicalJoin;
+import org.apache.calcite.rel.logical.LogicalProject;
+import org.apache.calcite.rel.metadata.RelMdUtil;
+import org.apache.calcite.rel.rules.FilterJoinRule;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexCorrelVariable;
+import org.apache.calcite.rex.RexFieldAccess;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+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.sql.SqlFunction;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.fun.SqlCountAggFunction;
+import org.apache.calcite.sql.fun.SqlSingleValueAggFunction;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.Bug;
+import org.apache.calcite.util.Holder;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.ReflectUtil;
+import org.apache.calcite.util.ReflectiveVisitDispatcher;
+import org.apache.calcite.util.ReflectiveVisitor;
+import org.apache.calcite.util.Util;
+import org.apache.calcite.util.mapping.Mappings;
+import org.apache.calcite.util.trace.CalciteTrace;
 
 import com.google.common.base.Supplier;
-import com.google.common.collect.*;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Multimap;
+import com.google.common.collect.Multimaps;
+import com.google.common.collect.Sets;
+import com.google.common.collect.SortedSetMultimap;
+
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import java.util.TreeSet;
+import java.util.logging.Level;
+import java.util.logging.Logger;
 
 /**
  * RelDecorrelator replaces all correlated expressions (corExp) in a relational
@@ -62,7 +121,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
   //~ Static fields/initializers ---------------------------------------------
 
   private static final Logger SQL2REL_LOGGER =
-      EigenbaseTrace.getSqlToRelTracer();
+      CalciteTrace.getSqlToRelTracer();
 
   //~ Instance fields --------------------------------------------------------
 
@@ -90,7 +149,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
   private final Map<RelNode, Map<Integer, Integer>>
   mapNewRelToMapOldToNewOutputPos = Maps.newHashMap();
 
-  private final HashSet<CorrelatorRel> generatedCorRels = Sets.newHashSet();
+  private final HashSet<Correlator> generatedCorRels = Sets.newHashSet();
 
   //~ Constructors -----------------------------------------------------------
 
@@ -112,7 +171,9 @@ public class RelDecorrelator implements ReflectiveVisitor {
    * <p>This is the main entry point to {@code RelDecorrelator}.
    *
    * @param rootRel Root node of the query
-   * @return Equivalent query with all {@link CorrelatorRel} instances removed
+   *
+   * @return Equivalent query with all
+   * {@link org.apache.calcite.rel.core.Correlator} instances removed
    */
   public static RelNode decorrelateQuery(RelNode rootRel) {
     final CorelMap corelMap = CorelMap.build(rootRel);
@@ -132,7 +193,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
     if (SQL2REL_LOGGER.isLoggable(Level.FINE)) {
       SQL2REL_LOGGER.fine(
           RelOptUtil.dumpPlan(
-              "Plan after removing CorrelatorRel",
+              "Plan after removing Correlator",
               newRootRel,
               false,
               SqlExplainLevel.EXPPLAN_ATTRIBUTES));
@@ -145,7 +206,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
     return newRootRel;
   }
 
-  private void setCurrent(RelNode root, CorrelatorRel corRel) {
+  private void setCurrent(RelNode root, Correlator corRel) {
     currentRel = corRel;
     if (corRel != null) {
       cm = CorelMap.build(Util.first(root, corRel));
@@ -157,7 +218,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
     HepProgram program = HepProgram.builder()
         .addRuleInstance(new AdjustProjectForCountAggregateRule(false))
         .addRuleInstance(new AdjustProjectForCountAggregateRule(true))
-        .addRuleInstance(PushFilterPastJoinRule.FILTER_ON_JOIN)
+        .addRuleInstance(FilterJoinRule.FILTER_ON_JOIN)
         .build();
 
     HepPlanner planner = createPlanner(program);
@@ -188,17 +249,17 @@ public class RelDecorrelator implements ReflectiveVisitor {
           cm.mapRefRelToCorVar.putAll(newNode,
               cm.mapRefRelToCorVar.get(oldNode));
         }
-        if (oldNode instanceof CorrelatorRel
-            && newNode instanceof CorrelatorRel) {
-          CorrelatorRel oldCor = (CorrelatorRel) oldNode;
+        if (oldNode instanceof Correlator
+            && newNode instanceof Correlator) {
+          Correlator oldCor = (Correlator) oldNode;
           for (Correlation c : oldCor.getCorrelations()) {
             if (cm.mapCorVarToCorRel.get(c) == oldNode) {
-              cm.mapCorVarToCorRel.put(c, (CorrelatorRel) newNode);
+              cm.mapCorVarToCorRel.put(c, (Correlator) newNode);
             }
           }
 
           if (generatedCorRels.contains(oldNode)) {
-            generatedCorRels.add((CorrelatorRel) newNode);
+            generatedCorRels.add((Correlator) newNode);
           }
         }
         return null;
@@ -306,27 +367,27 @@ public class RelDecorrelator implements ReflectiveVisitor {
   }
 
   /**
-   * Rewrite SortRel.
+   * Rewrite Sort.
    *
-   * @param rel SortRel to be rewritten
+   * @param rel Sort to be rewritten
    */
-  public void decorrelateRel(SortRel rel) {
+  public void decorrelateRel(Sort rel) {
     //
     // Rewrite logic:
     //
     // 1. change the collations field to reference the new input.
     //
 
-    // SortRel itself should not reference cor vars.
+    // Sort itself should not reference cor vars.
     assert !cm.mapRefRelToCorVar.containsKey(rel);
 
-    // SortRel only references field positions in collations field.
+    // Sort only references field positions in collations field.
     // The collations field in the newRel now need to refer to the
     // new output positions in its input.
     // Its output does not change the input ordering, so there's no
     // need to call propagateExpr.
 
-    RelNode oldChildRel = rel.getChild();
+    RelNode oldChildRel = rel.getInput();
 
     RelNode newChildRel = mapOldToNewRel.get(oldChildRel);
     if (newChildRel == null) {
@@ -346,8 +407,8 @@ public class RelDecorrelator implements ReflectiveVisitor {
     RelCollation oldCollation = rel.getCollation();
     RelCollation newCollation = RexUtil.apply(mapping, oldCollation);
 
-    SortRel newRel =
-        new SortRel(
+    Sort newRel =
+        new Sort(
             rel.getCluster(),
             rel.getCluster().traitSetOf(Convention.NONE).plus(newCollation),
             newChildRel,
@@ -357,29 +418,29 @@ public class RelDecorrelator implements ReflectiveVisitor {
 
     mapOldToNewRel.put(rel, newRel);
 
-    // SortRel does not change input ordering
+    // Sort does not change input ordering
     mapNewRelToMapOldToNewOutputPos.put(newRel, childMapOldToNewOutputPos);
   }
 
   /**
-   * Rewrite AggregateRel.
+   * Rewrites a {@link LogicalAggregate}.
    *
-   * @param rel the project rel to rewrite
+   * @param rel Aggregate to rewrite
    */
-  public void decorrelateRel(AggregateRel rel) {
+  public void decorrelateRel(LogicalAggregate rel) {
     //
     // Rewrite logic:
     //
     // 1. Permute the group by keys to the front.
-    // 2. If the child of an AggregateRel produces correlated variables,
+    // 2. If the child of an aggregate produces correlated variables,
     //    add them to the group list.
-    // 3. Change aggCalls to reference the new ProjectRel.
+    // 3. Change aggCalls to reference the new project.
     //
 
-    // AggregaterRel itself should not reference cor vars.
+    // Aggregate itself should not reference cor vars.
     assert !cm.mapRefRelToCorVar.containsKey(rel);
 
-    RelNode oldChildRel = rel.getChild();
+    RelNode oldChildRel = rel.getInput();
 
     RelNode newChildRel = mapOldToNewRel.get(oldChildRel);
     if (newChildRel == null) {
@@ -395,7 +456,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
     Map<Integer, Integer> mapNewChildToProjOutputPos = Maps.newHashMap();
     final int oldGroupKeyCount = rel.getGroupSet().cardinality();
 
-    // ProjectRel projects the original expressions,
+    // LogicalProject projects the original expressions,
     // plus any correlated variables the child wants to pass along.
     final List<Pair<RexNode, String>> projects = Lists.newArrayList();
 
@@ -447,7 +508,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
 
     assert newPos == newChildOutput.size();
 
-    // This ProjectRel will be what the old child maps to,
+    // This LogicalProject will be what the old child maps to,
     // replacing any previous mapping from old child).
     RelNode newProjectRel =
         RelOptUtil.createProject(newChildRel, projects, false);
@@ -482,11 +543,11 @@ public class RelDecorrelator implements ReflectiveVisitor {
           mapCorVarToOutputPos);
     }
 
-    // now it's time to rewrite AggregateRel
+    // now it's time to rewrite LogicalAggregate
     List<AggregateCall> newAggCalls = Lists.newArrayList();
     List<AggregateCall> oldAggCalls = rel.getAggCallList();
 
-    // AggregateRel.Call oldAggCall;
+    // LogicalAggregate.Call oldAggCall;
     int oldChildOutputFieldCount = oldChildRel.getRowType().getFieldCount();
     int newChildOutputFieldCount =
         newProjectRel.getRowType().getFieldCount();
@@ -517,39 +578,39 @@ public class RelDecorrelator implements ReflectiveVisitor {
           newChildOutputFieldCount + i);
     }
 
-    AggregateRel newAggregateRel =
-        new AggregateRel(
+    LogicalAggregate newAggregate =
+        new LogicalAggregate(
             rel.getCluster(),
             newProjectRel,
             BitSets.range(newGroupKeyCount),
             newAggCalls);
 
-    mapOldToNewRel.put(rel, newAggregateRel);
+    mapOldToNewRel.put(rel, newAggregate);
 
-    mapNewRelToMapOldToNewOutputPos.put(newAggregateRel, combinedMap);
+    mapNewRelToMapOldToNewOutputPos.put(newAggregate, combinedMap);
 
     if (produceCorVar) {
-      // AggregateRel does not change input ordering so corVars will be
+      // LogicalAggregate does not change input ordering so corVars will be
       // located at the same position as the input newProjectRel.
       mapNewRelToMapCorVarToOutputPos.put(
-          newAggregateRel,
+          newAggregate,
           mapCorVarToOutputPos);
     }
   }
 
   /**
-   * Rewrite ProjectRel.
+   * Rewrite LogicalProject.
    *
    * @param rel the project rel to rewrite
    */
-  public void decorrelateRel(ProjectRel rel) {
+  public void decorrelateRel(LogicalProject rel) {
     //
     // Rewrite logic:
     //
     // 1. Pass along any correlated variables coming from the child.
     //
 
-    RelNode oldChildRel = rel.getChild();
+    RelNode oldChildRel = rel.getInput();
 
     RelNode newChildRel = mapOldToNewRel.get(oldChildRel);
     if (newChildRel == null) {
@@ -568,22 +629,22 @@ public class RelDecorrelator implements ReflectiveVisitor {
     boolean produceCorVar =
         mapNewRelToMapCorVarToOutputPos.containsKey(newChildRel);
 
-    // ProjectRel projects the original expressions,
+    // LogicalProject projects the original expressions,
     // plus any correlated variables the child wants to pass along.
     final List<Pair<RexNode, String>> projects = Lists.newArrayList();
 
-    // If this ProjectRel has correlated reference, create value generator
+    // If this LogicalProject has correlated reference, create value generator
     // and produce the correlated variables in the new output.
     if (cm.mapRefRelToCorVar.containsKey(rel)) {
       decorrelateInputWithValueGenerator(rel);
 
-      // The old child should be mapped to the JoinRel created by
+      // The old child should be mapped to the LogicalJoin created by
       // rewriteInputWithValueGenerator().
       newChildRel = mapOldToNewRel.get(oldChildRel);
       produceCorVar = true;
     }
 
-    // ProjectRel projects the original expressions
+    // LogicalProject projects the original expressions
     int newPos;
     for (newPos = 0; newPos < oldProj.size(); newPos++) {
       projects.add(
@@ -713,7 +774,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
           resultRel = distinctRel;
         } else {
           resultRel =
-              new JoinRel(
+              new LogicalJoin(
                   cluster,
                   resultRel,
                   distinctRel,
@@ -794,7 +855,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
 
     final Set<String> variablesStopped = Collections.emptySet();
     RelNode joinRel =
-        new JoinRel(
+        new LogicalJoin(
             rel.getCluster(),
             newLeftChildRel,
             valueGenRel,
@@ -805,35 +866,35 @@ public class RelDecorrelator implements ReflectiveVisitor {
     mapOldToNewRel.put(oldChildRel, joinRel);
     mapNewRelToMapCorVarToOutputPos.put(joinRel, mapCorVarToOutputPos);
 
-    // JoinRel or FilterRel does not change the old input ordering. All
+    // LogicalJoin or LogicalFilter does not change the old input ordering. All
     // input fields from newLeftInput(i.e. the original input to the old
-    // FilterRel) are in the output and in the same position.
+    // LogicalFilter) are in the output and in the same position.
     mapNewRelToMapOldToNewOutputPos.put(joinRel, childMapOldToNewOutputPos);
   }
 
   /**
-   * Rewrite FilterRel.
+   * Rewrite LogicalFilter.
    *
    * @param rel the filter rel to rewrite
    */
-  public void decorrelateRel(FilterRel rel) {
+  public void decorrelateRel(LogicalFilter rel) {
     //
     // Rewrite logic:
     //
-    // 1. If a FilterRel references a correlated field in its filter
-    // condition, rewrite the FilterRel to be
-    //   FilterRel
-    //     JoinRel(cross product)
+    // 1. If a LogicalFilter references a correlated field in its filter
+    // condition, rewrite the LogicalFilter to be
+    //   LogicalFilter
+    //     LogicalJoin(cross product)
     //       OriginalFilterInput
     //       ValueGenerator(produces distinct sets of correlated variables)
     // and rewrite the correlated fieldAccess in the filter condition to
-    // reference the JoinRel output.
+    // reference the LogicalJoin output.
     //
-    // 2. If FilterRel does not reference correlated variables, simply
+    // 2. If LogicalFilter does not reference correlated variables, simply
     // rewrite the filter condition using new input.
     //
 
-    RelNode oldChildRel = rel.getChild();
+    RelNode oldChildRel = rel.getInput();
 
     RelNode newChildRel = mapOldToNewRel.get(oldChildRel);
     if (newChildRel == null) {
@@ -848,12 +909,12 @@ public class RelDecorrelator implements ReflectiveVisitor {
     boolean produceCorVar =
         mapNewRelToMapCorVarToOutputPos.containsKey(newChildRel);
 
-    // If this FilterRel has correlated reference, create value generator
+    // If this LogicalFilter has correlated reference, create value generator
     // and produce the correlated variables in the new output.
     if (cm.mapRefRelToCorVar.containsKey(rel)) {
       decorrelateInputWithValueGenerator(rel);
 
-      // The old child should be mapped to the newly created JoinRel by
+      // The old child should be mapped to the newly created LogicalJoin by
       // rewriteInputWithValueGenerator().
       newChildRel = mapOldToNewRel.get(oldChildRel);
       produceCorVar = true;
@@ -883,11 +944,11 @@ public class RelDecorrelator implements ReflectiveVisitor {
   }
 
   /**
-   * Rewrite CorrelatorRel into a left outer join.
+   * Rewrite Correlator into a left outer join.
    *
-   * @param rel CorrelatorRel
+   * @param rel Correlator
    */
-  public void decorrelateRel(CorrelatorRel rel) {
+  public void decorrelateRel(Correlator rel) {
     //
     // Rewrite logic:
     //
@@ -897,7 +958,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
     // joined later with the CorrelatorRels that produce them.
     //
 
-    // the right input to CorrelatorRel should produce correlated variables
+    // the right input to Correlator should produce correlated variables
     RelNode oldLeftRel = rel.getInputs().get(0);
     RelNode oldRightRel = rel.getInputs().get(1);
 
@@ -1004,7 +1065,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
 
     final Set<String> variablesStopped = Collections.emptySet();
     RelNode newRel =
-        new JoinRel(
+        new LogicalJoin(
             rel.getCluster(),
             newLeftRel,
             newRightRel,
@@ -1021,11 +1082,11 @@ public class RelDecorrelator implements ReflectiveVisitor {
   }
 
   /**
-   * Rewrite JoinRel.
+   * Rewrite LogicalJoin.
    *
-   * @param rel JoinRel
+   * @param rel LogicalJoin
    */
-  public void decorrelateRel(JoinRel rel) {
+  public void decorrelateRel(LogicalJoin rel) {
     //
     // Rewrite logic:
     //
@@ -1056,7 +1117,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
 
     final Set<String> variablesStopped = Collections.emptySet();
     RelNode newRel =
-        new JoinRel(
+        new LogicalJoin(
             rel.getCluster(),
             newLeftRel,
             newRightRel,
@@ -1158,27 +1219,27 @@ public class RelDecorrelator implements ReflectiveVisitor {
   }
 
   /**
-   * Pull projRel above the joinRel from its RHS input. Enforce nullability
+   * Pull projRel above the join from its RHS input. Enforce nullability
    * for join output.
    *
-   * @param joinRel          Join
+   * @param join          Join
    * @param projRel          the original projRel as the RHS input of the join.
    * @param nullIndicatorPos Position of null indicator
-   * @return the subtree with the new ProjectRel at the root
+   * @return the subtree with the new LogicalProject at the root
    */
   private RelNode projectJoinOutputWithNullability(
-      JoinRel joinRel,
-      ProjectRel projRel,
+      LogicalJoin join,
+      LogicalProject projRel,
       int nullIndicatorPos) {
-    RelDataTypeFactory typeFactory = joinRel.getCluster().getTypeFactory();
-    RelNode leftInputRel = joinRel.getLeft();
-    JoinRelType joinType = joinRel.getJoinType();
+    RelDataTypeFactory typeFactory = join.getCluster().getTypeFactory();
+    RelNode leftInputRel = join.getLeft();
+    JoinRelType joinType = join.getJoinType();
 
     RexInputRef nullIndicator =
         new RexInputRef(
             nullIndicatorPos,
             typeFactory.createTypeWithNullability(
-                joinRel.getRowType().getFieldList().get(nullIndicatorPos)
+                join.getRowType().getFieldList().get(nullIndicatorPos)
                     .getType(),
                 true));
 
@@ -1211,7 +1272,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
     }
 
     RelNode newProjRel =
-        RelOptUtil.createProject(joinRel, newProjExprs, false);
+        RelOptUtil.createProject(join, newProjExprs, false);
 
     return newProjRel;
   }
@@ -1221,14 +1282,14 @@ public class RelDecorrelator implements ReflectiveVisitor {
    * for join output.
    *
    * @param corRel  Correlator
-   * @param projRel the original ProjectRel as the RHS input of the join
+   * @param projRel the original LogicalProject as the RHS input of the join
    * @param isCount Positions which are calls to the <code>COUNT</code>
    *                aggregation function
-   * @return the subtree with the new ProjectRel at the root
+   * @return the subtree with the new LogicalProject at the root
    */
   private RelNode aggregateCorrelatorOutput(
-      CorrelatorRel corRel,
-      ProjectRel projRel,
+      Correlator corRel,
+      LogicalProject projRel,
       Set<Integer> isCount) {
     RelNode leftInputRel = corRel.getLeft();
     JoinRelType joinType = corRel.getJoinType();
@@ -1264,28 +1325,27 @@ public class RelDecorrelator implements ReflectiveVisitor {
   }
 
   /**
-   * Checks whether the correlations in projRel and filterRel are related to
+   * Checks whether the correlations in projRel and filter are related to
    * the correlated variables provided by corRel.
    *
-   * @param corRel             Correlator
-   * @param projRel            the original ProjectRel as the RHS input of the
-   *                           join
-   * @param filterRel          Filter
+   * @param corRel    Correlator
+   * @param projRel   The original Project as the RHS input of the join
+   * @param filter    Filter
    * @param correlatedJoinKeys Correlated join keys
    * @return true if filter and proj only references corVar provided by corRel
    */
   private boolean checkCorVars(
-      CorrelatorRel corRel,
-      ProjectRel projRel,
-      FilterRel filterRel,
+      Correlator corRel,
+      LogicalProject projRel,
+      LogicalFilter filter,
       List<RexFieldAccess> correlatedJoinKeys) {
-    if (filterRel != null) {
+    if (filter != null) {
       assert correlatedJoinKeys != null;
 
       // check that all correlated refs in the filter condition are
       // used in the join(as field access).
       Set<Correlation> corVarInFilter =
-          Sets.newHashSet(cm.mapRefRelToCorVar.get(filterRel));
+          Sets.newHashSet(cm.mapRefRelToCorVar.get(filter));
 
       for (RexFieldAccess correlatedJoinKey : correlatedJoinKeys) {
         corVarInFilter.remove(
@@ -1298,7 +1358,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
 
       // Check that the correlated variables referenced in these
       // comparisons do come from the correlatorRel.
-      corVarInFilter.addAll(cm.mapRefRelToCorVar.get(filterRel));
+      corVarInFilter.addAll(cm.mapRefRelToCorVar.get(filter));
 
       for (Correlation corVar : corVarInFilter) {
         if (cm.mapCorVarToCorRel.get(corVar) != corRel) {
@@ -1326,7 +1386,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
    *
    * @param corRel Correlator
    */
-  private void removeCorVarFromTree(CorrelatorRel corRel) {
+  private void removeCorVarFromTree(Correlator corRel) {
     for (Correlation c : Lists.newArrayList(cm.mapCorVarToCorRel.keySet())) {
       if (cm.mapCorVarToCorRel.get(c) == corRel) {
         cm.mapCorVarToCorRel.remove(c);
@@ -1339,7 +1399,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
    *
    * @param childRel        Child relational expression
    * @param additionalExprs Additional expressions and names
-   * @return the new ProjectRel
+   * @return the new LogicalProject
    */
   private RelNode createProjectWithAdditionalExprs(
       RelNode childRel,
@@ -1360,6 +1420,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
 
   //~ Inner Classes ----------------------------------------------------------
 
+  /** Visitor that decorrelates. */
   private class DecorrelateRelVisitor extends RelVisitor {
     private final ReflectiveVisitDispatcher<RelDecorrelator, RelNode>
     dispatcher =
@@ -1391,6 +1452,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
     }
   }
 
+  /** Shuttle that decorrelates. */
   private class DecorrelateRexShuttle extends RexShuttle {
     // override RexShuttle
     public RexNode visitFieldAccess(RexFieldAccess fieldAccess) {
@@ -1450,6 +1512,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
     }
   }
 
+  /** Shuttle that removes correlations. */
   private class RemoveCorrelationRexShuttle extends RexShuttle {
     RexBuilder rexBuilder;
     RelDataTypeFactory typeFactory;
@@ -1553,7 +1616,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
         Correlation corVar = cm.mapFieldAccessToCorVar.get(fieldAccess);
 
         // corVar offset should point to the leftInput of currentRel,
-        // which is the CorrelatorRel.
+        // which is the Correlator.
         RexNode newRexNode =
             new RexInputRef(corVar.getOffset(), fieldAccess.getType());
 
@@ -1574,14 +1637,14 @@ public class RelDecorrelator implements ReflectiveVisitor {
 
     // override RexShuttle
     public RexNode visitInputRef(RexInputRef inputRef) {
-      if ((currentRel != null) && (currentRel instanceof CorrelatorRel)) {
+      if ((currentRel != null) && (currentRel instanceof Correlator)) {
         // if this rel references corVar
         // and now it needs to be rewritten
-        // it must have been pulled above the CorrelatorRel
+        // it must have been pulled above the Correlator
         // replace the input ref to account for the LHS of the
-        // CorrelatorRel
+        // Correlator
         int leftInputFieldCount =
-            ((CorrelatorRel) currentRel).getLeft().getRowType()
+            ((Correlator) currentRel).getLeft().getRowType()
                 .getFieldCount();
         RelDataType newType = inputRef.getType();
 
@@ -1688,16 +1751,16 @@ public class RelDecorrelator implements ReflectiveVisitor {
     public RemoveSingleAggregateRule() {
       super(
           operand(
-              AggregateRel.class,
+              LogicalAggregate.class,
               operand(
-                  ProjectRel.class,
-                  operand(AggregateRel.class, any()))));
+                  LogicalProject.class,
+                  operand(LogicalAggregate.class, any()))));
     }
 
     public void onMatch(RelOptRuleCall call) {
-      AggregateRel singleAggRel = call.rel(0);
-      ProjectRel projRel = call.rel(1);
-      AggregateRel aggRel = call.rel(2);
+      LogicalAggregate singleAggRel = call.rel(0);
+      LogicalProject projRel = call.rel(1);
+      LogicalAggregate aggRel = call.rel(2);
 
       // check singleAggRel is single_value agg
       if ((!singleAggRel.getGroupSet().isEmpty())
@@ -1736,24 +1799,22 @@ public class RelDecorrelator implements ReflectiveVisitor {
     }
   }
 
+  /** Planner rule that removes correlations for scalar projects. */
   private final class RemoveCorrelationForScalarProjectRule extends RelOptRule {
     public RemoveCorrelationForScalarProjectRule() {
       super(
-          operand(
-              CorrelatorRel.class,
+          operand(Correlator.class,
               operand(RelNode.class, any()),
-              operand(
-                  AggregateRel.class,
-                  operand(
-                      ProjectRel.class,
+              operand(LogicalAggregate.class,
+                  operand(LogicalProject.class,
                       operand(RelNode.class, any())))));
     }
 
     public void onMatch(RelOptRuleCall call) {
-      CorrelatorRel corRel = call.rel(0);
+      Correlator corRel = call.rel(0);
       RelNode leftInputRel = call.rel(1);
-      AggregateRel aggRel = call.rel(2);
-      ProjectRel projRel = call.rel(3);
+      LogicalAggregate aggRel = call.rel(2);
+      LogicalProject projRel = call.rel(3);
       RelNode rightInputRel = call.rel(4);
       RelOptCluster cluster = corRel.getCluster();
 
@@ -1765,8 +1826,8 @@ public class RelDecorrelator implements ReflectiveVisitor {
       //
       // CorrelateRel(left correlation, condition = true)
       //   LeftInputRel
-      //   AggregateRel (groupby (0) single_value())
-      //     ProjectRel-A (may reference coVar)
+      //   LogicalAggregate (groupby (0) single_value())
+      //     LogicalProject-A (may reference coVar)
       //       RightInputRel
       JoinRelType joinType = corRel.getJoinType();
       RexNode joinCond = corRel.getCondition();
@@ -1792,19 +1853,19 @@ public class RelDecorrelator implements ReflectiveVisitor {
 
       int nullIndicatorPos;
 
-      if ((rightInputRel instanceof FilterRel)
+      if ((rightInputRel instanceof LogicalFilter)
           && cm.mapRefRelToCorVar.containsKey(rightInputRel)) {
         // rightInputRel has this shape:
         //
-        //       FilterRel (references corvar)
+        //       LogicalFilter (references corvar)
         //         FilterInputRel
 
         // If rightInputRel is a filter and contains correlated
         // reference, make sure the correlated keys in the filter
         // condition forms a unique key of the RHS.
 
-        FilterRel filterRel = (FilterRel) rightInputRel;
-        rightInputRel = filterRel.getChild();
+        LogicalFilter filter = (LogicalFilter) rightInputRel;
+        rightInputRel = filter.getInput();
 
         assert rightInputRel instanceof HepRelVertex;
         rightInputRel = ((HepRelVertex) rightInputRel).getCurrentRel();
@@ -1814,7 +1875,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
           return;
         }
 
-        // extract the correlation out of the filterRel
+        // extract the correlation out of the filter
 
         // First breaking up the filter conditions into equality
         // comparisons between rightJoinKeys(from the original
@@ -1824,7 +1885,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
         List<RexNode> tmpRightJoinKeys = Lists.newArrayList();
         List<RexNode> correlatedJoinKeys = Lists.newArrayList();
         RelOptUtil.splitCorrelatedFilterCondition(
-            filterRel,
+            filter,
             tmpRightJoinKeys,
             correlatedJoinKeys,
             false);
@@ -1848,10 +1909,9 @@ public class RelDecorrelator implements ReflectiveVisitor {
         if (!RelMdUtil.areColumnsDefinitelyUniqueWhenNullsFiltered(
             rightInputRel,
             rightJoinKeys)) {
-          SQL2REL_LOGGER.fine(
-              rightJoinKeys.toString()
-                  + "are not unique keys for "
-                  + rightInputRel.toString());
+          SQL2REL_LOGGER.fine(rightJoinKeys.toString()
+              + "are not unique keys for "
+              + rightInputRel.toString());
           return;
         }
 
@@ -1861,21 +1921,21 @@ public class RelDecorrelator implements ReflectiveVisitor {
         List<RexFieldAccess> correlatedKeyList =
             visitor.getFieldAccessList();
 
-        if (!checkCorVars(corRel, projRel, filterRel, correlatedKeyList)) {
+        if (!checkCorVars(corRel, projRel, filter, correlatedKeyList)) {
           return;
         }
 
         // Change the plan to this structure.
         // Note that the aggregateRel is removed.
         //
-        // ProjectRel-A' (replace corvar to input ref from the JoinRel)
-        //   JoinRel (replace corvar to input ref from LeftInputRel)
+        // LogicalProject-A' (replace corvar to input ref from the LogicalJoin)
+        //   LogicalJoin (replace corvar to input ref from LeftInputRel)
         //     LeftInputRel
         //     RightInputRel(oreviously FilterInputRel)
 
         // Change the filter condition into a join condition
         joinCond =
-            removeCorrelationExpr(filterRel.getCondition(), false);
+            removeCorrelationExpr(filter.getCondition(), false);
 
         nullIndicatorPos =
             leftInputRel.getRowType().getFieldCount()
@@ -1892,11 +1952,11 @@ public class RelDecorrelator implements ReflectiveVisitor {
 
         // Change the plan to this structure.
         //
-        // ProjectRel-A' (replace corvar to input ref from JoinRel)
-        //   JoinRel (left, condition = true)
+        // LogicalProject-A' (replace corvar to input ref from LogicalJoin)
+        //   LogicalJoin (left, condition = true)
         //     LeftInputRel
-        //     AggregateRel(groupby(0), single_value(0), s_v(1)....)
-        //       ProjectRel-B (everything from input plus literal true)
+        //     LogicalAggregate(groupby(0), single_value(0), s_v(1)....)
+        //       LogicalProject-B (everything from input plus literal true)
         //         ProjInputRel
 
         // make the new projRel to provide a null indicator
@@ -1921,8 +1981,8 @@ public class RelDecorrelator implements ReflectiveVisitor {
       }
 
       // make the new join rel
-      JoinRel joinRel =
-          new JoinRel(
+      LogicalJoin join =
+          new LogicalJoin(
               corRel.getCluster(),
               leftInputRel,
               rightInputRel,
@@ -1931,7 +1991,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
               ImmutableSet.<String>of());
 
       RelNode newProjRel =
-          projectJoinOutputWithNullability(joinRel, projRel, nullIndicatorPos);
+          projectJoinOutputWithNullability(join, projRel, nullIndicatorPos);
 
       call.transformTo(newProjRel);
 
@@ -1939,28 +1999,29 @@ public class RelDecorrelator implements ReflectiveVisitor {
     }
   }
 
+  /** Planner rule that removes correlations for scalar aggregates. */
   private final class RemoveCorrelationForScalarAggregateRule
       extends RelOptRule {
     public RemoveCorrelationForScalarAggregateRule() {
       super(
           operand(
-              CorrelatorRel.class,
+              Correlator.class,
               operand(RelNode.class, any()),
               operand(
-                  ProjectRel.class,
+                  LogicalProject.class,
                   operand(
-                      AggregateRel.class,
+                      LogicalAggregate.class,
                       operand(
-                          ProjectRel.class,
+                          LogicalProject.class,
                           operand(RelNode.class, any()))))));
     }
 
     public void onMatch(RelOptRuleCall call) {
-      CorrelatorRel corRel = call.rel(0);
+      Correlator corRel = call.rel(0);
       RelNode leftInputRel = call.rel(1);
-      ProjectRel aggOutputProjRel = call.rel(2);
-      AggregateRel aggRel = call.rel(3);
-      ProjectRel aggInputProjRel = call.rel(4);
+      LogicalProject aggOutputProjRel = call.rel(2);
+      LogicalAggregate aggRel = call.rel(3);
+      LogicalProject aggInputProjRel = call.rel(4);
       RelNode rightInputRel = call.rel(5);
       RelOptCluster cluster = corRel.getCluster();
 
@@ -1972,9 +2033,9 @@ public class RelDecorrelator implements ReflectiveVisitor {
       //
       // CorrelateRel(left correlation, condition = true)
       //   LeftInputRel
-      //   ProjectRel-A (a RexNode)
-      //     AggregateRel (groupby (0), agg0(), agg1()...)
-      //       ProjectRel-B (references coVar)
+      //   LogicalProject-A (a RexNode)
+      //     LogicalAggregate (groupby (0), agg0(), agg1()...)
+      //       LogicalProject-B (references coVar)
       //         rightInputRel
 
       // check aggOutputProj projects only one expression
@@ -2011,14 +2072,14 @@ public class RelDecorrelator implements ReflectiveVisitor {
         }
       }
 
-      if ((rightInputRel instanceof FilterRel)
+      if ((rightInputRel instanceof LogicalFilter)
           && cm.mapRefRelToCorVar.containsKey(rightInputRel)) {
         // rightInputRel has this shape:
         //
-        //       FilterRel (references corvar)
+        //       LogicalFilter (references corvar)
         //         FilterInputRel
-        FilterRel filterRel = (FilterRel) rightInputRel;
-        rightInputRel = filterRel.getChild();
+        LogicalFilter filter = (LogicalFilter) rightInputRel;
+        rightInputRel = filter.getInput();
 
         assert rightInputRel instanceof HepRelVertex;
         rightInputRel = ((HepRelVertex) rightInputRel).getCurrentRel();
@@ -2029,7 +2090,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
         }
 
         // check filter condition type First extract the correlation out
-        // of the filterRel
+        // of the filter
 
         // First breaking up the filter conditions into equality
         // comparisons between rightJoinKeys(from the original
@@ -2039,7 +2100,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
         List<RexNode> rightJoinKeys = Lists.newArrayList();
         List<RexNode> tmpCorrelatedJoinKeys = Lists.newArrayList();
         RelOptUtil.splitCorrelatedFilterCondition(
-            filterRel,
+            filter,
             rightJoinKeys,
             tmpCorrelatedJoinKeys,
             true);
@@ -2070,17 +2131,16 @@ public class RelDecorrelator implements ReflectiveVisitor {
         if (!RelMdUtil.areColumnsDefinitelyUniqueWhenNullsFiltered(
             leftInputRel,
             correlatedInputRefJoinKeys)) {
-          SQL2REL_LOGGER.fine(
-              correlatedJoinKeys.toString()
-                  + "are not unique keys for "
-                  + leftInputRel.toString());
+          SQL2REL_LOGGER.fine(correlatedJoinKeys.toString()
+              + "are not unique keys for "
+              + leftInputRel.toString());
           return;
         }
 
         // check cor var references are valid
         if (!checkCorVars(corRel,
             aggInputProjRel,
-            filterRel,
+            filter,
             correlatedJoinKeys)) {
           return;
         }
@@ -2089,21 +2149,21 @@ public class RelDecorrelator implements ReflectiveVisitor {
         //
         // CorrelateRel(left correlation, condition = true)
         //   LeftInputRel
-        //   ProjectRel-A (a RexNode)
-        //     AggregateRel (groupby(0), agg0(),agg1()...)
-        //       ProjectRel-B (may reference coVar)
-        //         FilterRel (references corVar)
+        //   LogicalProject-A (a RexNode)
+        //     LogicalAggregate (groupby(0), agg0(),agg1()...)
+        //       LogicalProject-B (may reference coVar)
+        //         LogicalFilter (references corVar)
         //           RightInputRel (no correlated reference)
         //
 
         // to this plan:
         //
-        // ProjectRel-A' (all gby keys + rewritten nullable ProjExpr)
-        //   AggregateRel (groupby(all left input refs)
+        // LogicalProject-A' (all gby keys + rewritten nullable ProjExpr)
+        //   LogicalAggregate (groupby(all left input refs)
         //                 agg0(rewritten expression),
         //                 agg1()...)
-        //     ProjectRel-B' (rewriten original projected exprs)
-        //       JoinRel(replace corvar w/ input ref from LeftInputRel)
+        //     LogicalProject-B' (rewriten original projected exprs)
+        //       LogicalJoin(replace corvar w/ input ref from LeftInputRel)
         //         LeftInputRel
         //         RightInputRel
         //
@@ -2115,21 +2175,21 @@ public class RelDecorrelator implements ReflectiveVisitor {
         // projection list from the RHS for simplicity to avoid
         // searching for non-null fields.
         //
-        // ProjectRel-A' (all gby keys + rewritten nullable ProjExpr)
-        //   AggregateRel (groupby(all left input refs),
+        // LogicalProject-A' (all gby keys + rewritten nullable ProjExpr)
+        //   LogicalAggregate (groupby(all left input refs),
         //                 count(nullIndicator), other aggs...)
-        //     ProjectRel-B' (all left input refs plus
+        //     LogicalProject-B' (all left input refs plus
         //                    the rewritten original projected exprs)
-        //       JoinRel(replace corvar to input ref from LeftInputRel)
+        //       LogicalJoin(replace corvar to input ref from LeftInputRel)
         //         LeftInputRel
-        //         ProjectRel (everything from RightInputRel plus
+        //         LogicalProject (everything from RightInputRel plus
         //                     the nullIndicator "true")
         //           RightInputRel
         //
 
         // first change the filter condition into a join condition
         joinCond =
-            removeCorrelationExpr(filterRel.getCondition(), false);
+            removeCorrelationExpr(filter.getCondition(), false);
       } else if (cm.mapRefRelToCorVar.containsKey(aggInputProjRel)) {
         // check rightInputRel contains no correlation
         if (RelOptUtil.getVariablesUsed(rightInputRel).size() > 0) {
@@ -2158,20 +2218,20 @@ public class RelDecorrelator implements ReflectiveVisitor {
         //
         // CorrelateRel(left correlation, condition = true)
         //   LeftInputRel
-        //   ProjectRel-A (a RexNode)
-        //     AggregateRel (groupby(0), agg0(), agg1()...)
-        //       ProjectRel-B (references coVar)
+        //   LogicalProject-A (a RexNode)
+        //     LogicalAggregate (groupby(0), agg0(), agg1()...)
+        //       LogicalProject-B (references coVar)
         //         RightInputRel (no correlated reference)
         //
 
         // to this plan:
         //
-        // ProjectRel-A' (all gby keys + rewritten nullable ProjExpr)
-        //   AggregateRel (groupby(all left input refs)
+        // LogicalProject-A' (all gby keys + rewritten nullable ProjExpr)
+        //   LogicalAggregate (groupby(all left input refs)
         //                 agg0(rewritten expression),
         //                 agg1()...)
-        //     ProjectRel-B' (rewriten original projected exprs)
-        //       JoinRel (LOJ cond = true)
+        //     LogicalProject-B' (rewriten original projected exprs)
+        //       LogicalJoin (LOJ cond = true)
         //         LeftInputRel
         //         RightInputRel
         //
@@ -2183,14 +2243,14 @@ public class RelDecorrelator implements ReflectiveVisitor {
         // projection list from the RHS for simplicity to avoid
         // searching for non-null fields.
         //
-        // ProjectRel-A' (all gby keys + rewritten nullable ProjExpr)
-        //   AggregateRel (groupby(all left input refs),
+        // LogicalProject-A' (all gby keys + rewritten nullable ProjExpr)
+        //   LogicalAggregate (groupby(all left input refs),
         //                 count(nullIndicator), other aggs...)
-        //     ProjectRel-B' (all left input refs plus
+        //     LogicalProject-B' (all left input refs plus
         //                    the rewritten original projected exprs)
-        //       JoinRel(replace corvar to input ref from LeftInputRel)
+        //       LogicalJoin(replace corvar to input ref from LeftInputRel)
         //         LeftInputRel
-        //         ProjectRel (everything from RightInputRel plus
+        //         LogicalProject (everything from RightInputRel plus
         //                     the nullIndicator "true")
         //           RightInputRel
       } else {
@@ -2208,8 +2268,8 @@ public class RelDecorrelator implements ReflectiveVisitor {
                   Pair.<RexNode, String>of(rexBuilder.makeLiteral(true),
                       "nullIndicator")));
 
-      JoinRel joinRel =
-          new JoinRel(
+      LogicalJoin join =
+          new LogicalJoin(
               cluster,
               leftInputRel,
               rightInputRel,
@@ -2219,13 +2279,13 @@ public class RelDecorrelator implements ReflectiveVisitor {
 
       // To the consumer of joinOutputProjRel, nullIndicator is located
       // at the end
-      int nullIndicatorPos = joinRel.getRowType().getFieldCount() - 1;
+      int nullIndicatorPos = join.getRowType().getFieldCount() - 1;
 
       RexInputRef nullIndicator =
           new RexInputRef(
               nullIndicatorPos,
               cluster.getTypeFactory().createTypeWithNullability(
-                  joinRel.getRowType().getFieldList()
+                  join.getRowType().getFieldList()
                       .get(nullIndicatorPos).getType(),
                   true));
 
@@ -2247,11 +2307,11 @@ public class RelDecorrelator implements ReflectiveVisitor {
       }
 
       joinOutputProjExprs.add(
-          rexBuilder.makeInputRef(joinRel, nullIndicatorPos));
+          rexBuilder.makeInputRef(join, nullIndicatorPos));
 
       RelNode joinOutputProjRel =
           RelOptUtil.createProject(
-              joinRel,
+              join,
               joinOutputProjExprs,
               null);
 
@@ -2287,8 +2347,8 @@ public class RelDecorrelator implements ReflectiveVisitor {
 
       BitSet groupSet =
           BitSets.range(groupCount);
-      AggregateRel newAggRel =
-          new AggregateRel(
+      LogicalAggregate newAggRel =
+          new LogicalAggregate(
               cluster,
               joinOutputProjRel,
               groupSet,
@@ -2329,27 +2389,28 @@ public class RelDecorrelator implements ReflectiveVisitor {
   // REVIEW jvs 29-Oct-2007:  Shouldn't it also be incorporating
   // the flavor attribute into the description?
 
+  /** Planner rule that adjusts projects when counts are added. */
   private final class AdjustProjectForCountAggregateRule extends RelOptRule {
     final boolean flavor;
 
     public AdjustProjectForCountAggregateRule(boolean flavor) {
       super(
           flavor
-              ? operand(CorrelatorRel.class,
+              ? operand(Correlator.class,
                   operand(RelNode.class, any()),
-                      operand(ProjectRel.class,
-                          operand(AggregateRel.class, any())))
-              : operand(CorrelatorRel.class,
+                      operand(LogicalProject.class,
+                          operand(LogicalAggregate.class, any())))
+              : operand(Correlator.class,
                   operand(RelNode.class, any()),
-                      operand(AggregateRel.class, any())));
+                      operand(LogicalAggregate.class, any())));
       this.flavor = flavor;
     }
 
     public void onMatch(RelOptRuleCall call) {
-      CorrelatorRel corRel = call.rel(0);
+      Correlator corRel = call.rel(0);
       RelNode leftInputRel = call.rel(1);
-      ProjectRel aggOutputProjRel;
-      AggregateRel aggRel;
+      LogicalProject aggOutputProjRel;
+      LogicalAggregate aggRel;
       if (flavor) {
         aggOutputProjRel = call.rel(2);
         aggRel = call.rel(3);
@@ -2364,7 +2425,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
           projects.add(RexInputRef.of2(projects.size(), fields));
         }
         aggOutputProjRel =
-            (ProjectRel) RelOptUtil.createProject(
+            (LogicalProject) RelOptUtil.createProject(
                 aggRel,
                 projects,
                 false);
@@ -2374,10 +2435,10 @@ public class RelDecorrelator implements ReflectiveVisitor {
 
     private void onMatch2(
         RelOptRuleCall call,
-        CorrelatorRel corRel,
+        Correlator corRel,
         RelNode leftInputRel,
-        ProjectRel aggOutputProjRel,
-        AggregateRel aggRel) {
+        LogicalProject aggOutputProjRel,
+        LogicalAggregate aggRel) {
       RelOptCluster cluster = corRel.getCluster();
 
       if (generatedCorRels.contains(corRel)) {
@@ -2394,8 +2455,8 @@ public class RelDecorrelator implements ReflectiveVisitor {
       //
       // CorrelateRel(left correlation, condition = true)
       //   LeftInputRel
-      //   ProjectRel-A (a RexNode)
-      //     AggregateRel (groupby (0), agg0(), agg1()...)
+      //   LogicalProject-A (a RexNode)
+      //     LogicalAggregate (groupby (0), agg0(), agg1()...)
 
       // check aggOutputProj projects only one expression
       List<RexNode> aggOutputProjExprs = aggOutputProjRel.getProjects();
@@ -2431,12 +2492,12 @@ public class RelDecorrelator implements ReflectiveVisitor {
       //
       // Project-A' (all LHS plus transformed original projections,
       //             replacing references to count() with case statement)
-      //   CorrelatorRel(left correlation, condition = true)
+      //   Correlator(left correlation, condition = true)
       //     LeftInputRel
-      //     AggregateRel (groupby (0), agg0(), agg1()...)
+      //     LogicalAggregate (groupby (0), agg0(), agg1()...)
       //
-      CorrelatorRel newCorRel =
-          new CorrelatorRel(
+      Correlator newCorRel =
+          new Correlator(
               cluster,
               leftInputRel,
               aggRel,
@@ -2465,8 +2526,8 @@ public class RelDecorrelator implements ReflectiveVisitor {
     }
   }
 
-  /** A map of the locations of {@link org.eigenbase.rel.CorrelatorRel} in
-   * a tree of {@link RelNode}s.
+  /** A map of the locations of {@link org.apache.calcite.rel.core.Correlator}
+   * in a tree of {@link RelNode}s.
    *
    * <p>It is used to drive the decorrelation process.
    * Treat it as immutable; rebuild if you modify the tree.
@@ -2487,20 +2548,19 @@ public class RelDecorrelator implements ReflectiveVisitor {
    * </ol> */
   private static class CorelMap {
     private final Multimap<RelNode, Correlation> mapRefRelToCorVar;
-    private final SortedMap<Correlation, CorrelatorRel> mapCorVarToCorRel;
+    private final SortedMap<Correlation, Correlator> mapCorVarToCorRel;
     private final Map<RexFieldAccess, Correlation> mapFieldAccessToCorVar;
 
     // TODO: create immutable copies of all maps
     private CorelMap(Multimap<RelNode, Correlation> mapRefRelToCorVar,
-        SortedMap<Correlation, CorrelatorRel> mapCorVarToCorRel,
+        SortedMap<Correlation, Correlator> mapCorVarToCorRel,
         Map<RexFieldAccess, Correlation> mapFieldAccessToCorVar) {
       this.mapRefRelToCorVar = mapRefRelToCorVar;
       this.mapCorVarToCorRel = mapCorVarToCorRel;
       this.mapFieldAccessToCorVar = mapFieldAccessToCorVar;
     }
 
-    @Override
-    public String toString() {
+    @Override public String toString() {
       return "mapRefRelToCorVar=" + mapRefRelToCorVar
           + "\nmapCorVarToCorRel=" + mapCorVarToCorRel
           + "\nmapFieldAccessToCorVar=" + mapFieldAccessToCorVar
@@ -2517,14 +2577,15 @@ public class RelDecorrelator implements ReflectiveVisitor {
     }
 
     @Override public int hashCode() {
-      return Util.hashV(mapRefRelToCorVar, mapCorVarToCorRel,
+      return com.google.common.base.Objects.hashCode(mapRefRelToCorVar,
+          mapCorVarToCorRel,
           mapFieldAccessToCorVar);
     }
 
     /** Creates a CorelMap with given contents. */
     public static CorelMap of(
         SortedSetMultimap<RelNode, Correlation> mapRefRelToCorVar,
-        SortedMap<Correlation, CorrelatorRel> mapCorVarToCorRel,
+        SortedMap<Correlation, Correlator> mapCorVarToCorRel,
         Map<RexFieldAccess, Correlation> mapFieldAccessToCorVar) {
       return new CorelMap(mapRefRelToCorVar, mapCorVarToCorRel,
           mapFieldAccessToCorVar);
@@ -2532,8 +2593,8 @@ public class RelDecorrelator implements ReflectiveVisitor {
 
     /** Creates a CorelMap by iterating over a {@link RelNode} tree. */
     public static CorelMap build(RelNode rel) {
-      final SortedMap<Correlation, CorrelatorRel> mapCorVarToCorRel =
-          new TreeMap<Correlation, CorrelatorRel>();
+      final SortedMap<Correlation, Correlator> mapCorVarToCorRel =
+          new TreeMap<Correlation, Correlator>();
 
       final SortedSetMultimap<RelNode, Correlation> mapRefRelToCorVar =
           Multimaps.newSortedSetMultimap(
@@ -2552,18 +2613,16 @@ public class RelDecorrelator implements ReflectiveVisitor {
 
       final Holder<Integer> offset = Holder.of(0);
       final RelShuttleImpl shuttle = new RelShuttleImpl() {
-        @Override
-        public RelNode visit(JoinRel join) {
+        @Override public RelNode visit(LogicalJoin join) {
           return visitJoin(join);
         }
 
-        @Override
-        protected RelNode visitChild(RelNode parent, int i, RelNode child) {
+        @Override protected RelNode visitChild(RelNode parent, int i,
+            RelNode child) {
           return super.visitChild(parent, i, stripHep(child));
         }
 
-        @Override
-        public RelNode visit(CorrelatorRel correlator) {
+        @Override public RelNode visit(Correlator correlator) {
           for (Correlation c : correlator.getCorrelations()) {
             mapNameToCorVar.put("$cor" + c.getId(), c);
             mapCorVarToCorRel.put(c, correlator);
@@ -2571,7 +2630,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
           return visitJoin(correlator);
         }
 
-        private JoinRelBase visitJoin(JoinRelBase join) {
+        private Join visitJoin(Join join) {
           join.getCondition().accept(rexVisitor(join));
           final int x = offset.get();
           visitChild(join, 0, join.getLeft());
@@ -2581,14 +2640,12 @@ public class RelDecorrelator implements ReflectiveVisitor {
           return join;
         }
 
-        @Override
-        public RelNode visit(final FilterRel filter) {
+        @Override public RelNode visit(final LogicalFilter filter) {
           filter.getCondition().accept(rexVisitor(filter));
           return super.visit(filter);
         }
 
-        @Override
-        public RelNode visit(ProjectRel project) {
+        @Override public RelNode visit(LogicalProject project) {
           for (RexNode node : project.getProjects()) {
             node.accept(rexVisitor(project));
           }
@@ -2597,8 +2654,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
 
         private RexVisitorImpl<Void> rexVisitor(final RelNode rel) {
           return new RexVisitorImpl<Void>(true) {
-            @Override
-            public Void visitFieldAccess(RexFieldAccess fieldAccess) {
+            @Override public Void visitFieldAccess(RexFieldAccess fieldAccess) {
               final RexNode ref = fieldAccess.getReferenceExpr();
               if (ref instanceof RexCorrelVariable) {
                 final RexCorrelVariable var = (RexCorrelVariable) ref;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 8a78add..075def1 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/RelFieldTrimmer.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/RelFieldTrimmer.java
@@ -14,28 +14,61 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql2rel;
-
-import java.math.BigDecimal;
-import java.util.*;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.rel.rules.RemoveTrivialProjectRule;
-import org.eigenbase.rel.rules.SemiJoinRel;
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.rex.*;
-import org.eigenbase.sql.validate.SqlValidator;
-import org.eigenbase.util.*;
-import org.eigenbase.util.mapping.*;
-
-import net.hydromatic.linq4j.Ord;
-
-import net.hydromatic.optiq.util.BitSets;
+package org.apache.calcite.sql2rel;
+
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Aggregate;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.RelFactories;
+import org.apache.calcite.rel.core.SemiJoin;
+import org.apache.calcite.rel.core.SetOp;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.rel.logical.LogicalTableFunctionScan;
+import org.apache.calcite.rel.logical.LogicalTableModify;
+import org.apache.calcite.rel.logical.LogicalValues;
+import org.apache.calcite.rel.rules.ProjectRemoveRule;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rel.type.RelDataTypeImpl;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexPermuteInputsShuttle;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.rex.RexVisitor;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.Bug;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.ReflectUtil;
+import org.apache.calcite.util.ReflectiveVisitor;
+import org.apache.calcite.util.Util;
+import org.apache.calcite.util.mapping.IntPair;
+import org.apache.calcite.util.mapping.Mapping;
+import org.apache.calcite.util.mapping.MappingType;
+import org.apache.calcite.util.mapping.Mappings;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.Collections;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Set;
+
 /**
  * Transformer that walks over a tree of relational expressions, replacing each
  * {@link RelNode} with a 'slimmed down' relational expression that projects
@@ -247,7 +280,7 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
   }
 
   /**
-   * Visit method, per {@link org.eigenbase.util.ReflectiveVisitor}.
+   * Visit method, per {@link org.apache.calcite.util.ReflectiveVisitor}.
    *
    * <p>This method is invoked reflectively, so there may not be any apparent
    * calls to it. The class (or derived classes) may contain overloads of
@@ -276,15 +309,15 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
 
   /**
    * Variant of {@link #trimFields(RelNode, BitSet, Set)} for
-   * {@link ProjectRel}.
+   * {@link org.apache.calcite.rel.logical.LogicalProject}.
    */
   public TrimResult trimFields(
-      ProjectRelBase project,
+      Project project,
       BitSet fieldsUsed,
       Set<RelDataTypeField> extraFields) {
     final RelDataType rowType = project.getRowType();
     final int fieldCount = rowType.getFieldCount();
-    final RelNode input = project.getChild();
+    final RelNode input = project.getInput();
     final RelDataType inputRowType = input.getRowType();
 
     // Which fields are required from the input?
@@ -344,7 +377,7 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
             mapping);
 
     final RelNode newProject;
-    if (RemoveTrivialProjectRule.isIdentity(
+    if (ProjectRemoveRule.isIdentity(
         newProjectExprList,
         newRowType,
         newInput.getRowType())) {
@@ -384,16 +417,16 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
 
   /**
    * Variant of {@link #trimFields(RelNode, BitSet, Set)} for
-   * {@link FilterRel}.
+   * {@link org.apache.calcite.rel.logical.LogicalFilter}.
    */
   public TrimResult trimFields(
-      FilterRelBase filter,
+      Filter filter,
       BitSet fieldsUsed,
       Set<RelDataTypeField> extraFields) {
     final RelDataType rowType = filter.getRowType();
     final int fieldCount = rowType.getFieldCount();
     final RexNode conditionExpr = filter.getCondition();
-    final RelNode input = filter.getChild();
+    final RelNode input = filter.getInput();
 
     // We use the fields used by the consumer, plus any fields used in the
     // filter.
@@ -436,16 +469,16 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
 
   /**
    * Variant of {@link #trimFields(RelNode, BitSet, Set)} for
-   * {@link SortRel}.
+   * {@link org.apache.calcite.rel.core.Sort}.
    */
   public TrimResult trimFields(
-      SortRel sort,
+      Sort sort,
       BitSet fieldsUsed,
       Set<RelDataTypeField> extraFields) {
     final RelDataType rowType = sort.getRowType();
     final int fieldCount = rowType.getFieldCount();
     final RelCollation collation = sort.getCollation();
-    final RelNode input = sort.getChild();
+    final RelNode input = sort.getInput();
 
     // We use the fields used by the consumer, plus any fields used as sort
     // keys.
@@ -485,10 +518,10 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
 
   /**
    * Variant of {@link #trimFields(RelNode, BitSet, Set)} for
-   * {@link JoinRel}.
+   * {@link org.apache.calcite.rel.logical.LogicalJoin}.
    */
   public TrimResult trimFields(
-      JoinRelBase join,
+      Join join,
       BitSet fieldsUsed,
       Set<RelDataTypeField> extraFields) {
     final int fieldCount = join.getSystemFieldList().size()
@@ -603,7 +636,7 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
         conditionExpr.accept(shuttle);
 
     final RelNode newJoin;
-    if (join instanceof SemiJoinRel) {
+    if (join instanceof SemiJoin) {
       newJoin = semiJoinFactory.createSemiJoin(newInputs.get(0),
           newInputs.get(1), newConditionExpr);
       // For SemiJoins only map fields from the left-side
@@ -630,10 +663,10 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
 
   /**
    * Variant of {@link #trimFields(RelNode, BitSet, Set)} for
-   * {@link SetOpRel} (including UNION and UNION ALL).
+   * {@link org.apache.calcite.rel.core.SetOp} (including UNION and UNION ALL).
    */
   public TrimResult trimFields(
-      SetOpRel setOp,
+      SetOp setOp,
       BitSet fieldsUsed,
       Set<RelDataTypeField> extraFields) {
     final RelDataType rowType = setOp.getRowType();
@@ -698,10 +731,10 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
 
   /**
    * Variant of {@link #trimFields(RelNode, BitSet, Set)} for
-   * {@link AggregateRel}.
+   * {@link org.apache.calcite.rel.logical.LogicalAggregate}.
    */
   public TrimResult trimFields(
-      AggregateRelBase aggregate,
+      Aggregate aggregate,
       BitSet fieldsUsed,
       Set<RelDataTypeField> extraFields) {
     // Fields:
@@ -808,10 +841,10 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
 
   /**
    * Variant of {@link #trimFields(RelNode, BitSet, Set)} for
-   * {@link TableModificationRel}.
+   * {@link org.apache.calcite.rel.logical.LogicalTableModify}.
    */
   public TrimResult trimFields(
-      TableModificationRel modifier,
+      LogicalTableModify modifier,
       BitSet fieldsUsed,
       Set<RelDataTypeField> extraFields) {
     // Ignore what consumer wants. We always project all columns.
@@ -819,7 +852,7 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
 
     final RelDataType rowType = modifier.getRowType();
     final int fieldCount = rowType.getFieldCount();
-    RelNode input = modifier.getChild();
+    RelNode input = modifier.getInput();
 
     // We want all fields from the child.
     final int inputFieldCount = input.getRowType().getFieldCount();
@@ -838,7 +871,7 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
           "Expected identity mapping, got " + inputMapping);
     }
 
-    TableModificationRel newModifier = modifier;
+    LogicalTableModify newModifier = modifier;
     if (newInput != input) {
       newModifier =
           modifier.copy(
@@ -854,10 +887,10 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
 
   /**
    * Variant of {@link #trimFields(RelNode, BitSet, Set)} for
-   * {@link TableFunctionRel}.
+   * {@link org.apache.calcite.rel.logical.LogicalTableFunctionScan}.
    */
   public TrimResult trimFields(
-      TableFunctionRel tabFun,
+      LogicalTableFunctionScan tabFun,
       BitSet fieldsUsed,
       Set<RelDataTypeField> extraFields) {
     final RelDataType rowType = tabFun.getRowType();
@@ -878,7 +911,7 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
       newInputs.add(trimResult.left);
     }
 
-    TableFunctionRel newTabFun = tabFun;
+    LogicalTableFunctionScan newTabFun = tabFun;
     if (!tabFun.getInputs().equals(newInputs)) {
       newTabFun = tabFun.copy(tabFun.getTraitSet(), newInputs);
     }
@@ -891,10 +924,10 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
 
   /**
    * Variant of {@link #trimFields(RelNode, BitSet, Set)} for
-   * {@link org.eigenbase.rel.ValuesRel}.
+   * {@link org.apache.calcite.rel.logical.LogicalValues}.
    */
   public TrimResult trimFields(
-      ValuesRel values,
+      LogicalValues values,
       BitSet fieldsUsed,
       Set<RelDataTypeField> extraFields) {
     final RelDataType rowType = values.getRowType();
@@ -926,8 +959,8 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
     final RelDataType newRowType =
         RelOptUtil.permute(values.getCluster().getTypeFactory(), rowType,
             mapping);
-    final ValuesRel newValues =
-        new ValuesRel(values.getCluster(), newRowType, newTuples);
+    final LogicalValues newValues =
+        new LogicalValues(values.getCluster(), newRowType, newTuples);
     return new TrimResult(newValues, mapping);
   }
 
@@ -946,10 +979,10 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
 
   /**
    * Variant of {@link #trimFields(RelNode, BitSet, Set)} for
-   * {@link org.eigenbase.rel.TableAccessRel}.
+   * {@link org.apache.calcite.rel.logical.LogicalTableScan}.
    */
   public TrimResult trimFields(
-      final TableAccessRelBase tableAccessRel,
+      final TableScan tableAccessRel,
       BitSet fieldsUsed,
       Set<RelDataTypeField> extraFields) {
     final int fieldCount = tableAccessRel.getRowType().getFieldCount();
@@ -966,13 +999,13 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
     // pretend that one field is used.
     if (fieldsUsed.cardinality() == 0) {
       RelNode input = newTableAccessRel;
-      if (input instanceof ProjectRelBase) {
+      if (input instanceof Project) {
         // The table has implemented the project in the obvious way - by
         // creating project with 0 fields. Strip it away, and create our own
         // project with one field.
-        ProjectRelBase project = (ProjectRelBase) input;
+        Project project = (Project) input;
         if (project.getRowType().getFieldCount() == 0) {
-          input = project.getChild();
+          input = project.getInput();
         }
       }
       return dummyProject(fieldCount, input);
@@ -991,7 +1024,7 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
    * of the current relational expression.
    *
    * <p>The mapping is a
-   * {@link org.eigenbase.util.mapping.Mappings.SourceMapping}, which means
+   * {@link org.apache.calcite.util.mapping.Mappings.SourceMapping}, which means
    * that no column can be used more than once, and some columns are not used.
    * {@code columnsUsed.getSource(i)} returns the source of the i'th output
    * field.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql2rel/RelStructuredTypeFlattener.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql2rel/RelStructuredTypeFlattener.java b/core/src/main/java/org/apache/calcite/sql2rel/RelStructuredTypeFlattener.java
index eacbbac..d30531e 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/RelStructuredTypeFlattener.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/RelStructuredTypeFlattener.java
@@ -14,22 +14,71 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql2rel;
-
-import java.util.*;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.rex.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.fun.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.util.*;
-import org.eigenbase.util.mapping.Mappings;
+package org.apache.calcite.sql2rel;
+
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelVisitor;
+import org.apache.calcite.rel.core.Collect;
+import org.apache.calcite.rel.core.Correlation;
+import org.apache.calcite.rel.core.Correlator;
+import org.apache.calcite.rel.core.Sample;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.core.Uncollect;
+import org.apache.calcite.rel.logical.LogicalAggregate;
+import org.apache.calcite.rel.logical.LogicalCalc;
+import org.apache.calcite.rel.logical.LogicalFilter;
+import org.apache.calcite.rel.logical.LogicalIntersect;
+import org.apache.calcite.rel.logical.LogicalJoin;
+import org.apache.calcite.rel.logical.LogicalMinus;
+import org.apache.calcite.rel.logical.LogicalOneRow;
+import org.apache.calcite.rel.logical.LogicalProject;
+import org.apache.calcite.rel.logical.LogicalTableFunctionScan;
+import org.apache.calcite.rel.logical.LogicalTableModify;
+import org.apache.calcite.rel.logical.LogicalTableScan;
+import org.apache.calcite.rel.logical.LogicalUnion;
+import org.apache.calcite.rel.logical.LogicalValues;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexCorrelVariable;
+import org.apache.calcite.rex.RexFieldAccess;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexLocalRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexProgramBuilder;
+import org.apache.calcite.rex.RexShuttle;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlOperator;
+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.ReflectUtil;
+import org.apache.calcite.util.ReflectiveVisitDispatcher;
+import org.apache.calcite.util.ReflectiveVisitor;
+import org.apache.calcite.util.Util;
+import org.apache.calcite.util.mapping.Mappings;
 
 import com.google.common.base.Function;
-import com.google.common.collect.*;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.SortedSetMultimap;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.SortedSet;
 
 // TODO jvs 10-Feb-2005:  factor out generic rewrite helper, with the
 // ability to map between old and new rels and field ordinals.  Also,
@@ -48,14 +97,14 @@ import com.google.common.collect.*;
  * unflattened tree looks like:
  *
  * <pre><code>
- * ProjectRel(C2=[$1], A2=[$0.A2])
- *   TableAccessRel(table=[T])
+ * LogicalProject(C2=[$1], A2=[$0.A2])
+ *   LogicalTableScan(table=[T])
  * </code></pre>
  *
  * After flattening, the resulting tree looks like
  *
  * <pre><code>
- * ProjectRel(C2=[$3], A2=[$2])
+ * LogicalProject(C2=[$3], A2=[$2])
  *   FtrsIndexScanRel(table=[T], index=[clustered])
  * </code></pre>
  *
@@ -106,13 +155,13 @@ public class RelStructuredTypeFlattener implements ReflectiveVisitor {
   }
 
   public void updateRelInMap(
-      SortedMap<Correlation, CorrelatorRel> mapCorVarToCorRel) {
+      SortedMap<Correlation, Correlator> mapCorVarToCorRel) {
     for (Correlation corVar : mapCorVarToCorRel.keySet()) {
-      CorrelatorRel oldRel = mapCorVarToCorRel.get(corVar);
+      Correlator oldRel = mapCorVarToCorRel.get(corVar);
       if (oldToNewRelMap.containsKey(oldRel)) {
         RelNode newRel = oldToNewRelMap.get(oldRel);
-        assert newRel instanceof CorrelatorRel;
-        mapCorVarToCorRel.put(corVar, (CorrelatorRel) newRel);
+        assert newRel instanceof Correlator;
+        mapCorVarToCorRel.put(corVar, (Correlator) newRel);
       }
     }
   }
@@ -295,21 +344,21 @@ public class RelStructuredTypeFlattener implements ReflectiveVisitor {
     return exp.accept(shuttle);
   }
 
-  public void rewriteRel(TableModificationRel rel) {
-    TableModificationRel newRel =
-        new TableModificationRel(
+  public void rewriteRel(LogicalTableModify rel) {
+    LogicalTableModify newRel =
+        new LogicalTableModify(
             rel.getCluster(),
             rel.getTable(),
             rel.getCatalogReader(),
-            getNewForOldRel(rel.getChild()),
+            getNewForOldRel(rel.getInput()),
             rel.getOperation(),
             rel.getUpdateColumnList(),
             true);
     setNewForOldRel(rel, newRel);
   }
 
-  public void rewriteRel(AggregateRel rel) {
-    RelDataType inputType = rel.getChild().getRowType();
+  public void rewriteRel(LogicalAggregate rel) {
+    RelDataType inputType = rel.getInput().getRowType();
     for (RelDataTypeField field : inputType.getFieldList()) {
       if (field.getType().isStruct()) {
         // TODO jvs 10-Feb-2005
@@ -320,9 +369,9 @@ public class RelStructuredTypeFlattener implements ReflectiveVisitor {
     rewriteGeneric(rel);
   }
 
-  public void rewriteRel(SortRel rel) {
+  public void rewriteRel(Sort rel) {
     RelCollation oldCollation = rel.getCollation();
-    final RelNode oldChild = rel.getChild();
+    final RelNode oldChild = rel.getInput();
     final RelNode newChild = getNewForOldRel(oldChild);
     final Mappings.TargetMapping mapping =
         getNewForOldInputMapping(oldChild);
@@ -338,8 +387,8 @@ public class RelStructuredTypeFlattener implements ReflectiveVisitor {
       }
     }
     RelCollation newCollation = RexUtil.apply(mapping, oldCollation);
-    SortRel newRel =
-        new SortRel(
+    Sort newRel =
+        new Sort(
             rel.getCluster(),
             rel.getCluster().traitSetOf(Convention.NONE).plus(newCollation),
             newChild,
@@ -349,17 +398,17 @@ public class RelStructuredTypeFlattener implements ReflectiveVisitor {
     setNewForOldRel(rel, newRel);
   }
 
-  public void rewriteRel(FilterRel rel) {
+  public void rewriteRel(LogicalFilter rel) {
     RelNode newRel =
         RelOptUtil.createFilter(
-            getNewForOldRel(rel.getChild()),
+            getNewForOldRel(rel.getInput()),
             flattenFieldAccesses(rel.getCondition()));
     setNewForOldRel(rel, newRel);
   }
 
-  public void rewriteRel(JoinRel rel) {
-    JoinRel newRel =
-        new JoinRel(
+  public void rewriteRel(LogicalJoin rel) {
+    LogicalJoin newRel =
+        new LogicalJoin(
             rel.getCluster(),
             getNewForOldRel(rel.getLeft()),
             getNewForOldRel(rel.getRight()),
@@ -369,7 +418,7 @@ public class RelStructuredTypeFlattener implements ReflectiveVisitor {
     setNewForOldRel(rel, newRel);
   }
 
-  public void rewriteRel(CorrelatorRel rel) {
+  public void rewriteRel(Correlator rel) {
     final List<Correlation> newCorrelations =
         new ArrayList<Correlation>();
     for (Correlation c : rel.getCorrelations()) {
@@ -384,8 +433,8 @@ public class RelStructuredTypeFlattener implements ReflectiveVisitor {
               c.getId(),
               getNewForOldInput(c.getOffset())));
     }
-    CorrelatorRel newRel =
-        new CorrelatorRel(
+    Correlator newRel =
+        new Correlator(
             rel.getCluster(),
             getNewForOldRel(rel.getLeft()),
             getNewForOldRel(rel.getRight()),
@@ -395,47 +444,47 @@ public class RelStructuredTypeFlattener implements ReflectiveVisitor {
     setNewForOldRel(rel, newRel);
   }
 
-  public void rewriteRel(CollectRel rel) {
+  public void rewriteRel(Collect rel) {
     rewriteGeneric(rel);
   }
 
-  public void rewriteRel(UncollectRel rel) {
+  public void rewriteRel(Uncollect rel) {
     rewriteGeneric(rel);
   }
 
-  public void rewriteRel(IntersectRel rel) {
+  public void rewriteRel(LogicalIntersect rel) {
     rewriteGeneric(rel);
   }
 
-  public void rewriteRel(MinusRel rel) {
+  public void rewriteRel(LogicalMinus rel) {
     rewriteGeneric(rel);
   }
 
-  public void rewriteRel(UnionRel rel) {
+  public void rewriteRel(LogicalUnion rel) {
     rewriteGeneric(rel);
   }
 
-  public void rewriteRel(OneRowRel rel) {
+  public void rewriteRel(LogicalOneRow rel) {
     rewriteGeneric(rel);
   }
 
-  public void rewriteRel(ValuesRel rel) {
+  public void rewriteRel(LogicalValues rel) {
     // NOTE jvs 30-Apr-2006:  UDT instances require invocation
     // of a constructor method, which can't be represented
-    // by the tuples stored in a ValuesRel, so we don't have
+    // by the tuples stored in a LogicalValues, so we don't have
     // to worry about them here.
     rewriteGeneric(rel);
   }
 
-  public void rewriteRel(TableFunctionRel rel) {
+  public void rewriteRel(LogicalTableFunctionScan rel) {
     rewriteGeneric(rel);
   }
 
-  public void rewriteRel(SamplingRel rel) {
+  public void rewriteRel(Sample rel) {
     rewriteGeneric(rel);
   }
 
-  public void rewriteRel(ProjectRel rel) {
+  public void rewriteRel(LogicalProject rel) {
     final List<RexNode> flattenedExpList = new ArrayList<RexNode>();
     final List<String> flattenedFieldNameList = new ArrayList<String>();
     List<String> fieldNames = rel.getRowType().getFieldNames();
@@ -447,15 +496,15 @@ public class RelStructuredTypeFlattener implements ReflectiveVisitor {
         flattenedFieldNameList);
     RelNode newRel =
         RelOptUtil.createProject(
-            getNewForOldRel(rel.getChild()),
+            getNewForOldRel(rel.getInput()),
             flattenedExpList,
             flattenedFieldNameList);
     setNewForOldRel(rel, newRel);
   }
 
-  public void rewriteRel(CalcRel rel) {
+  public void rewriteRel(LogicalCalc rel) {
     // Translate the child.
-    final RelNode newChild = getNewForOldRel(rel.getChild());
+    final RelNode newChild = getNewForOldRel(rel.getInput());
 
     final RelOptCluster cluster = rel.getCluster();
     RexProgramBuilder programBuilder =
@@ -502,8 +551,8 @@ public class RelStructuredTypeFlattener implements ReflectiveVisitor {
     RexProgram newProgram = programBuilder.getProgram();
 
     // Create a new calc relational expression.
-    CalcRel newRel =
-        new CalcRel(
+    LogicalCalc newRel =
+        new LogicalCalc(
             cluster,
             rel.getTraitSet(),
             newChild,
@@ -653,7 +702,7 @@ public class RelStructuredTypeFlattener implements ReflectiveVisitor {
         || (call.isA(SqlKind.NEW_SPECIFICATION));
   }
 
-  public void rewriteRel(TableAccessRel rel) {
+  public void rewriteRel(LogicalTableScan rel) {
     RelNode newRel =
         rel.getTable().toRel(toRelContext);
 
@@ -662,15 +711,18 @@ public class RelStructuredTypeFlattener implements ReflectiveVisitor {
 
   //~ Inner Interfaces -------------------------------------------------------
 
+  /** Mix-in interface for relational expressions that know how to
+   * flatten themselves. */
   public interface SelfFlatteningRel extends RelNode {
     void flattenRel(RelStructuredTypeFlattener flattener);
   }
 
   //~ Inner Classes ----------------------------------------------------------
 
+  /** Visitor that flattens each relational expression in a tree. */
   private class RewriteRelVisitor extends RelVisitor {
     private final ReflectiveVisitDispatcher<RelStructuredTypeFlattener,
-        RelNode> dispatcher =
+            RelNode> dispatcher =
         ReflectUtil.createDispatcher(
             RelStructuredTypeFlattener.class,
             RelNode.class);
@@ -703,6 +755,7 @@ public class RelStructuredTypeFlattener implements ReflectiveVisitor {
     }
   }
 
+  /** Shuttle that rewrites scalar expressions. */
   private class RewriteRexShuttle extends RexShuttle {
     // override RexShuttle
     public RexNode visitInputRef(RexInputRef input) {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql2rel/SqlNodeToRexConverter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql2rel/SqlNodeToRexConverter.java b/core/src/main/java/org/apache/calcite/sql2rel/SqlNodeToRexConverter.java
index 662ee4b..c72a914 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/SqlNodeToRexConverter.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/SqlNodeToRexConverter.java
@@ -14,10 +14,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql2rel;
+package org.apache.calcite.sql2rel;
 
-import org.eigenbase.rex.*;
-import org.eigenbase.sql.*;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlIntervalQualifier;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlNode;
 
 /**
  * Converts expressions from {@link SqlNode} to {@link RexNode}.
@@ -33,8 +37,8 @@ public interface SqlNodeToRexConverter {
       SqlCall call);
 
   /**
-   * Converts a {@link SqlLiteral SQL literal} to a {@link RexLiteral REX
-   * literal}.
+   * Converts a {@link SqlLiteral SQL literal} to a
+   * {@link RexLiteral REX literal}.
    *
    * <p>The result is {@link RexNode}, not {@link RexLiteral} because if the
    * literal is NULL (or the boolean Unknown value), we make a <code>CAST(NULL

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql2rel/SqlNodeToRexConverterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql2rel/SqlNodeToRexConverterImpl.java b/core/src/main/java/org/apache/calcite/sql2rel/SqlNodeToRexConverterImpl.java
index 540fc7f..3504f29 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/SqlNodeToRexConverterImpl.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/SqlNodeToRexConverterImpl.java
@@ -14,20 +14,29 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql2rel;
-
-import java.math.*;
-import java.util.*;
-
-import org.eigenbase.reltype.*;
-import org.eigenbase.rex.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.sql.validate.*;
-import org.eigenbase.util.*;
-
-import net.hydromatic.avatica.ByteString;
+package org.apache.calcite.sql2rel;
+
+import org.apache.calcite.avatica.ByteString;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlIntervalLiteral;
+import org.apache.calcite.sql.SqlIntervalQualifier;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlTimeLiteral;
+import org.apache.calcite.sql.SqlTimestampLiteral;
+import org.apache.calcite.sql.parser.SqlParserUtil;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.util.BitString;
+import org.apache.calcite.util.NlsString;
+import org.apache.calcite.util.Util;
+
+import java.math.BigDecimal;
+import java.util.Calendar;
 
 /**
  * Standard implementation of {@link SqlNodeToRexConverter}.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql2rel/SqlRexContext.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql2rel/SqlRexContext.java b/core/src/main/java/org/apache/calcite/sql2rel/SqlRexContext.java
index 082b376..15ce0ee 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/SqlRexContext.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/SqlRexContext.java
@@ -14,12 +14,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql2rel;
+package org.apache.calcite.sql2rel;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.rex.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.validate.*;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexRangeRef;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlSelect;
+import org.apache.calcite.sql.validate.SqlValidator;
 
 /**
  * Contains the context necessary for a {@link SqlRexConvertlet} to convert a
@@ -50,7 +55,7 @@ public interface SqlRexContext {
    * <p>Returns -1 if the query is not an aggregate query.</p>
    * @return 0 if the query is implicitly GROUP BY (), -1 if the query is not
    * and aggregate query
-   * @see org.eigenbase.sql.SqlOperatorBinding#getGroupCount()
+   * @see org.apache.calcite.sql.SqlOperatorBinding#getGroupCount()
    */
   int getGroupCount();
 
@@ -60,8 +65,8 @@ public interface SqlRexContext {
   RexBuilder getRexBuilder();
 
   /**
-   * Returns the expression used to access a given IN or EXISTS {@link
-   * SqlSelect sub-query}.
+   * Returns the expression used to access a given IN or EXISTS
+   * {@link SqlSelect sub-query}.
    *
    * @param call IN or EXISTS expression
    * @return Expression used to access current row of sub-query

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/sql2rel/SqlRexConvertlet.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql2rel/SqlRexConvertlet.java b/core/src/main/java/org/apache/calcite/sql2rel/SqlRexConvertlet.java
index f011c4a..487f5dd 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/SqlRexConvertlet.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/SqlRexConvertlet.java
@@ -14,10 +14,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.sql2rel;
+package org.apache.calcite.sql2rel;
 
-import org.eigenbase.rex.*;
-import org.eigenbase.sql.*;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlNode;
 
 /**
  * Thunk which converts a {@link SqlNode} expression into a {@link RexNode}


[48/58] [abbrv] [partial] incubator-calcite git commit: [CALCITE-306] Standardize code style for "import package.*; "

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableRules.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableRules.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableRules.java
index 1384e53..a9c8dd8 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableRules.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableRules.java
@@ -14,3367 +14,87 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.rules.java;
+package org.apache.calcite.adapter.enumerable;
 
-import net.hydromatic.linq4j.*;
-import net.hydromatic.linq4j.expressions.*;
-import net.hydromatic.linq4j.expressions.Expression;
-import net.hydromatic.linq4j.function.*;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.util.trace.CalciteTrace;
 
-import net.hydromatic.optiq.*;
-import net.hydromatic.optiq.impl.interpreter.Interpreter;
-import net.hydromatic.optiq.impl.interpreter.Row;
-import net.hydromatic.optiq.impl.java.JavaTypeFactory;
-import net.hydromatic.optiq.jdbc.JavaTypeFactoryImpl;
-import net.hydromatic.optiq.prepare.OptiqPrepareImpl;
-import net.hydromatic.optiq.prepare.Prepare;
-import net.hydromatic.optiq.rules.java.impl.*;
-import net.hydromatic.optiq.runtime.SortedMultiMap;
-import net.hydromatic.optiq.util.BitSets;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.rel.convert.ConverterRule;
-import org.eigenbase.rel.metadata.RelColumnMapping;
-import org.eigenbase.rel.metadata.RelMetadataQuery;
-import org.eigenbase.rel.rules.EquiJoinRel;
-import org.eigenbase.rel.rules.SemiJoinRel;
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.rex.*;
-import org.eigenbase.sql.fun.SqlStdOperatorTable;
-import org.eigenbase.trace.EigenbaseTrace;
-import org.eigenbase.util.*;
-
-import com.google.common.base.Function;
-import com.google.common.collect.ImmutableList;
-
-import java.lang.reflect.*;
-import java.math.BigDecimal;
-import java.util.*;
 import java.util.logging.Logger;
 
 /**
  * Rules and relational operators for the
  * {@link EnumerableConvention enumerable calling convention}.
  */
-public class JavaRules {
-  protected static final Logger LOGGER = EigenbaseTrace.getPlannerTracer();
+public class EnumerableRules {
+  protected static final Logger LOGGER = CalciteTrace.getPlannerTracer();
 
   public static final boolean BRIDGE_METHODS = true;
 
-  private static final List<ParameterExpression> NO_PARAMS =
-      Collections.emptyList();
-
-  private static final List<Expression> NO_EXPRS =
-      Collections.emptyList();
-
   public static final RelOptRule ENUMERABLE_JOIN_RULE =
       new EnumerableJoinRule();
 
   public static final RelOptRule ENUMERABLE_SEMI_JOIN_RULE =
       new EnumerableSemiJoinRule();
 
-  public static final String[] LEFT_RIGHT = new String[]{"left", "right"};
-
-  private static final boolean B = false;
-
-  private JavaRules() {
-  }
-
-  private static class EnumerableJoinRule extends ConverterRule {
-    private EnumerableJoinRule() {
-      super(
-          JoinRel.class,
-          Convention.NONE,
-          EnumerableConvention.INSTANCE,
-          "EnumerableJoinRule");
-    }
-
-    @Override
-    public RelNode convert(RelNode rel) {
-      JoinRel join = (JoinRel) rel;
-      List<RelNode> newInputs = new ArrayList<RelNode>();
-      for (RelNode input : join.getInputs()) {
-        if (!(input.getConvention() instanceof EnumerableConvention)) {
-          input =
-              convert(
-                  input,
-                  input.getTraitSet()
-                      .replace(EnumerableConvention.INSTANCE));
-        }
-        newInputs.add(input);
-      }
-      final RelNode left = newInputs.get(0);
-      final RelNode right = newInputs.get(1);
-      final JoinInfo info = JoinInfo.of(left, right, join.getCondition());
-      if (!info.isEqui() && join.getJoinType() != JoinRelType.INNER) {
-        // EnumerableJoinRel only supports equi-join. We can put a filter on top
-        // if it is an inner join.
-        return null;
-      }
-      final RelOptCluster cluster = join.getCluster();
-      RelNode newRel;
-      try {
-        newRel = new EnumerableJoinRel(
-            cluster,
-            join.getTraitSet().replace(EnumerableConvention.INSTANCE),
-            left,
-            right,
-            info.getEquiCondition(left, right, cluster.getRexBuilder()),
-            info.leftKeys,
-            info.rightKeys,
-            join.getJoinType(),
-            join.getVariablesStopped());
-      } catch (InvalidRelException e) {
-        LOGGER.fine(e.toString());
-        return null;
-      }
-      if (!info.isEqui()) {
-        newRel = new EnumerableFilterRel(cluster, newRel.getTraitSet(),
-            newRel, info.getRemaining(cluster.getRexBuilder()));
-      }
-      return newRel;
-    }
-  }
-
-  /** Implementation of {@link org.eigenbase.rel.JoinRel} in
-   * {@link EnumerableConvention enumerable calling convention}. */
-  public static class EnumerableJoinRel
-      extends EquiJoinRel
-      implements EnumerableRel {
-    protected EnumerableJoinRel(
-        RelOptCluster cluster,
-        RelTraitSet traits,
-        RelNode left,
-        RelNode right,
-        RexNode condition,
-        ImmutableIntList leftKeys,
-        ImmutableIntList rightKeys,
-        JoinRelType joinType,
-        Set<String> variablesStopped)
-        throws InvalidRelException {
-      super(
-          cluster,
-          traits,
-          left,
-          right,
-          condition,
-          leftKeys,
-          rightKeys,
-          joinType,
-          variablesStopped);
-    }
-
-    @Override
-    public EnumerableJoinRel copy(RelTraitSet traitSet, RexNode condition,
-        RelNode left, RelNode right, JoinRelType joinType,
-        boolean semiJoinDone) {
-      final JoinInfo joinInfo = JoinInfo.of(left, right, condition);
-      assert joinInfo.isEqui();
-      try {
-        return new EnumerableJoinRel(getCluster(), traitSet, left, right,
-            condition, joinInfo.leftKeys, joinInfo.rightKeys, joinType,
-            variablesStopped);
-      } catch (InvalidRelException e) {
-        // Semantic error not possible. Must be a bug. Convert to
-        // internal error.
-        throw new AssertionError(e);
-      }
-    }
-
-    @Override
-    public RelOptCost computeSelfCost(RelOptPlanner planner) {
-      double rowCount = RelMetadataQuery.getRowCount(this);
-
-      // Joins can be flipped, and for many algorithms, both versions are viable
-      // and have the same cost. To make the results stable between versions of
-      // the planner, make one of the versions slightly more expensive.
-      switch (joinType) {
-      case RIGHT:
-        rowCount = addEpsilon(rowCount);
-        break;
-      default:
-        if (left.getId() > right.getId()) {
-          rowCount = addEpsilon(rowCount);
-        }
-      }
-
-      // Cheaper if the smaller number of rows is coming from the LHS.
-      // Model this by adding L log L to the cost.
-      final double rightRowCount = right.getRows();
-      final double leftRowCount = left.getRows();
-      if (Double.isInfinite(leftRowCount)) {
-        rowCount = leftRowCount;
-      } else {
-        rowCount += Util.nLogN(leftRowCount);
-      }
-      if (Double.isInfinite(rightRowCount)) {
-        rowCount = rightRowCount;
-      } else {
-        rowCount += rightRowCount;
-      }
-      return planner.getCostFactory().makeCost(rowCount, 0, 0);
-    }
-
-    private double addEpsilon(double d) {
-      assert d >= 0d;
-      final double d0 = d;
-      if (d < 10) {
-        // For small d, adding 1 would change the value significantly.
-        d *= 1.001d;
-        if (d != d0) {
-          return d;
-        }
-      }
-      // For medium d, add 1. Keeps integral values integral.
-      ++d;
-      if (d != d0) {
-        return d;
-      }
-      // For large d, adding 1 might not change the value. Add .1%.
-      // If d is NaN, this still will probably not change the value. That's OK.
-      d *= 1.001d;
-      return d;
-    }
-
-    public Result implement(EnumerableRelImplementor implementor, Prefer pref) {
-      BlockBuilder builder = new BlockBuilder();
-      final Result leftResult =
-          implementor.visitChild(this, 0, (EnumerableRel) left, pref);
-      Expression leftExpression =
-          builder.append(
-              "left", leftResult.block);
-      final Result rightResult =
-          implementor.visitChild(this, 1, (EnumerableRel) right, pref);
-      Expression rightExpression =
-          builder.append(
-              "right", rightResult.block);
-      final PhysType physType =
-          PhysTypeImpl.of(
-              implementor.getTypeFactory(), getRowType(), pref.preferArray());
-      final PhysType keyPhysType =
-          leftResult.physType.project(
-              leftKeys, JavaRowFormat.LIST);
-      return implementor.result(
-          physType,
-          builder.append(
-              Expressions.call(
-                  leftExpression,
-                  BuiltinMethod.JOIN.method,
-                  Expressions.list(
-                      rightExpression,
-                      leftResult.physType.generateAccessor(leftKeys),
-                      rightResult.physType.generateAccessor(rightKeys),
-                      generateSelector(
-                          physType,
-                          ImmutableList.of(
-                              leftResult.physType, rightResult.physType)))
-                      .append(
-                          Util.first(keyPhysType.comparer(),
-                              Expressions.constant(null)))
-                      .append(Expressions.constant(
-                          joinType.generatesNullsOnLeft()))
-                      .append(Expressions.constant(
-                          joinType.generatesNullsOnRight())))).toBlock());
-    }
-
-    Expression generateSelector(PhysType physType,
-        List<PhysType> inputPhysTypes) {
-      // A parameter for each input.
-      final List<ParameterExpression> parameters =
-          new ArrayList<ParameterExpression>();
-
-      // Generate all fields.
-      final List<Expression> expressions =
-          new ArrayList<Expression>();
-      for (Ord<PhysType> ord : Ord.zip(inputPhysTypes)) {
-        final PhysType inputPhysType =
-            ord.e.makeNullable(joinType.generatesNullsOn(ord.i));
-        final ParameterExpression parameter =
-            Expressions.parameter(inputPhysType.getJavaRowType(),
-                LEFT_RIGHT[ord.i]);
-        parameters.add(parameter);
-        final int fieldCount = inputPhysType.getRowType().getFieldCount();
-        for (int i = 0; i < fieldCount; i++) {
-          Expression expression =
-              inputPhysType.fieldReference(parameter, i,
-                  physType.getJavaFieldType(i));
-          if (joinType.generatesNullsOn(ord.i)) {
-            expression =
-                Expressions.condition(
-                    Expressions.equal(parameter, Expressions.constant(null)),
-                    Expressions.constant(null),
-                    expression);
-          }
-          expressions.add(expression);
-        }
-      }
-      return Expressions.lambda(
-          Function2.class,
-          physType.record(expressions),
-          parameters);
-    }
-  }
-
-  private static class EnumerableSemiJoinRule extends ConverterRule {
-    private EnumerableSemiJoinRule() {
-      super(SemiJoinRel.class,
-          Convention.NONE,
-          EnumerableConvention.INSTANCE,
-          "EnumerableSemiJoinRule");
-    }
-
-    @Override
-    public RelNode convert(RelNode rel) {
-      final SemiJoinRel semiJoin = (SemiJoinRel) rel;
-      List<RelNode> newInputs = new ArrayList<RelNode>();
-      for (RelNode input : semiJoin.getInputs()) {
-        if (!(input.getConvention() instanceof EnumerableConvention)) {
-          input =
-              convert(input,
-                  input.getTraitSet().replace(EnumerableConvention.INSTANCE));
-        }
-        newInputs.add(input);
-      }
-      try {
-        return new EnumerableSemiJoinRel(
-            semiJoin.getCluster(),
-            semiJoin.getTraitSet().replace(EnumerableConvention.INSTANCE),
-            newInputs.get(0),
-            newInputs.get(1),
-            semiJoin.getCondition(),
-            semiJoin.leftKeys,
-            semiJoin.rightKeys);
-      } catch (InvalidRelException e) {
-        LOGGER.fine(e.toString());
-        return null;
-      }
-    }
-  }
-
-  /** Implementation of {@link org.eigenbase.rel.rules.SemiJoinRel} in
-   * {@link EnumerableConvention enumerable calling convention}. */
-  public static class EnumerableSemiJoinRel
-      extends SemiJoinRel
-      implements EnumerableRel {
-    protected EnumerableSemiJoinRel(
-        RelOptCluster cluster,
-        RelTraitSet traits,
-        RelNode left,
-        RelNode right,
-        RexNode condition,
-        ImmutableIntList leftKeys,
-        ImmutableIntList rightKeys)
-        throws InvalidRelException {
-      super(cluster, traits, left, right, condition, leftKeys, rightKeys);
-    }
-
-    @Override
-    public SemiJoinRel copy(RelTraitSet traitSet, RexNode condition,
-        RelNode left, RelNode right, JoinRelType joinType,
-        boolean semiJoinDone) {
-      assert joinType == JoinRelType.INNER;
-      final JoinInfo joinInfo = JoinInfo.of(left, right, condition);
-      assert joinInfo.isEqui();
-      try {
-        return new EnumerableSemiJoinRel(getCluster(), traitSet, left, right,
-            condition, joinInfo.leftKeys, joinInfo.rightKeys);
-      } catch (InvalidRelException e) {
-        // Semantic error not possible. Must be a bug. Convert to
-        // internal error.
-        throw new AssertionError(e);
-      }
-    }
-
-    @Override
-    public RelOptCost computeSelfCost(RelOptPlanner planner) {
-      double rowCount = RelMetadataQuery.getRowCount(this);
-
-      // Right-hand input is the "build", and hopefully small, input.
-      final double rightRowCount = right.getRows();
-      final double leftRowCount = left.getRows();
-      if (Double.isInfinite(leftRowCount)) {
-        rowCount = leftRowCount;
-      } else {
-        rowCount += Util.nLogN(leftRowCount);
-      }
-      if (Double.isInfinite(rightRowCount)) {
-        rowCount = rightRowCount;
-      } else {
-        rowCount += rightRowCount;
-      }
-      return planner.getCostFactory().makeCost(rowCount, 0, 0).multiplyBy(.01d);
-    }
-
-    public Result implement(EnumerableRelImplementor implementor, Prefer pref) {
-      BlockBuilder builder = new BlockBuilder();
-      final Result leftResult =
-          implementor.visitChild(this, 0, (EnumerableRel) left, pref);
-      Expression leftExpression =
-          builder.append(
-              "left", leftResult.block);
-      final Result rightResult =
-          implementor.visitChild(this, 1, (EnumerableRel) right, pref);
-      Expression rightExpression =
-          builder.append(
-              "right", rightResult.block);
-      final PhysType physType = leftResult.physType;
-      return implementor.result(
-          physType,
-          builder.append(
-              Expressions.call(
-                  BuiltinMethod.SEMI_JOIN.method,
-                  Expressions.list(
-                      leftExpression,
-                      rightExpression,
-                      leftResult.physType.generateAccessor(leftKeys),
-                      rightResult.physType.generateAccessor(rightKeys))))
-              .toBlock());
-    }
-  }
-
-  /**
-   * Utilities for generating programs in the Enumerable (functional)
-   * style.
-   */
-  public static class EnumUtil {
-    /** Declares a method that overrides another method. */
-    public static MethodDeclaration overridingMethodDecl(Method method,
-        Iterable<ParameterExpression> parameters,
-        BlockStatement body) {
-      return Expressions.methodDecl(
-          method.getModifiers() & ~Modifier.ABSTRACT,
-          method.getReturnType(),
-          method.getName(),
-          parameters,
-          body);
-    }
-
-    static Type javaClass(
-        JavaTypeFactory typeFactory, RelDataType type) {
-      final Type clazz = typeFactory.getJavaClass(type);
-      return clazz instanceof Class ? clazz : Object[].class;
-    }
-
-    static Class javaRowClass(
-        JavaTypeFactory typeFactory, RelDataType type) {
-      if (type.isStruct() && type.getFieldCount() == 1) {
-        type = type.getFieldList().get(0).getType();
-      }
-      final Type clazz = typeFactory.getJavaClass(type);
-      return clazz instanceof Class ? (Class) clazz : Object[].class;
-    }
-
-    static List<Type> fieldTypes(
-        final JavaTypeFactory typeFactory,
-        final List<? extends RelDataType> inputTypes) {
-      return new AbstractList<Type>() {
-        public Type get(int index) {
-          return EnumUtil.javaClass(typeFactory, inputTypes.get(index));
-        }
-        public int size() {
-          return inputTypes.size();
-        }
-      };
-    }
-
-    static List<RelDataType> fieldRowTypes(
-        final RelDataType inputRowType,
-        final List<? extends RexNode> extraInputs,
-        final List<Integer> argList) {
-      final List<RelDataTypeField> inputFields = inputRowType.getFieldList();
-      return new AbstractList<RelDataType>() {
-        public RelDataType get(int index) {
-          final int arg = argList.get(index);
-          return arg < inputFields.size()
-              ? inputFields.get(arg).getType()
-              : extraInputs.get(arg - inputFields.size()).getType();
-        }
-        public int size() {
-          return argList.size();
-        }
-      };
-    }
-  }
-
-  /** Implementation of {@link org.eigenbase.rel.TableAccessRel} in
-   * {@link EnumerableConvention enumerable calling convention}. */
-  public static class EnumerableTableAccessRel
-      extends TableAccessRelBase
-      implements EnumerableRel {
-    private final Class elementType;
-
-    public EnumerableTableAccessRel(RelOptCluster cluster, RelTraitSet traitSet,
-        RelOptTable table, Class elementType) {
-      super(cluster, traitSet, table);
-      assert getConvention() instanceof EnumerableConvention;
-      this.elementType = elementType;
-    }
-
-    private Expression getExpression(PhysType physType) {
-      final Expression expression = table.getExpression(Queryable.class);
-      final Expression expression2 = toEnumerable(expression);
-      assert Types.isAssignableFrom(Enumerable.class, expression2.getType());
-      Expression expression3 = toRows(physType, expression2);
-      return expression3;
-    }
-
-    private Expression toEnumerable(Expression expression) {
-      final Type type = expression.getType();
-      if (Types.isArray(type)) {
-        if (Types.toClass(type).getComponentType().isPrimitive()) {
-          expression =
-              Expressions.call(BuiltinMethod.AS_LIST.method, expression);
-        }
-        return Expressions.call(BuiltinMethod.AS_ENUMERABLE.method, expression);
-      } else if (Types.isAssignableFrom(Iterable.class, type)
-          && !Types.isAssignableFrom(Enumerable.class, type)) {
-        return Expressions.call(BuiltinMethod.AS_ENUMERABLE2.method,
-            expression);
-      } else if (Types.isAssignableFrom(Queryable.class, type)) {
-        // Queryable extends Enumerable, but it's too "clever", so we call
-        // Queryable.asEnumerable so that operations such as take(int) will be
-        // evaluated directly.
-        return Expressions.call(expression,
-            BuiltinMethod.QUERYABLE_AS_ENUMERABLE.method);
-      }
-      return expression;
-    }
-
-    private Expression toRows(PhysType physType, Expression expression) {
-      final ParameterExpression row_ =
-          Expressions.parameter(elementType, "row");
-      List<Expression> expressionList = new ArrayList<Expression>();
-      final int fieldCount = table.getRowType().getFieldCount();
-      if (elementType == Row.class) {
-        // Convert Enumerable<Row> to Enumerable<SyntheticRecord>
-        for (int i = 0; i < fieldCount; i++) {
-          expressionList.add(
-              RexToLixTranslator.convert(
-                  Expressions.call(row_,
-                      BuiltinMethod.ROW_VALUE.method,
-                      Expressions.constant(i)),
-                  physType.getJavaFieldType(i)));
-        }
-      } else if (elementType == Object[].class
-          && rowType.getFieldCount() == 1) {
-        // Convert Enumerable<Object[]> to Enumerable<SyntheticRecord>
-        for (int i = 0; i < fieldCount; i++) {
-          expressionList.add(
-              RexToLixTranslator.convert(
-                  Expressions.arrayIndex(row_, Expressions.constant(i)),
-                  physType.getJavaFieldType(i)));
-        }
-      } else if (elementType == Object.class) {
-        if (!(physType.getJavaRowType()
-            instanceof JavaTypeFactoryImpl.SyntheticRecordType)) {
-          return expression;
-        }
-        expressionList.add(
-            RexToLixTranslator.convert(row_, physType.getJavaFieldType(0)));
-      } else {
-        return expression;
-      }
-      return Expressions.call(expression,
-          BuiltinMethod.SELECT.method,
-          Expressions.lambda(Function1.class, physType.record(expressionList),
-              row_));
-    }
-
-    private JavaRowFormat format() {
-      if (Object[].class.isAssignableFrom(elementType)) {
-        return JavaRowFormat.ARRAY;
-      } else {
-        return JavaRowFormat.CUSTOM;
-      }
-    }
-
-    @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
-      return new EnumerableTableAccessRel(getCluster(), traitSet, table,
-          elementType);
-    }
-
-    public Result implement(EnumerableRelImplementor implementor, Prefer pref) {
-      // Note that representation is ARRAY. This assumes that the table
-      // returns a Object[] for each record. Actually a Table<T> can
-      // return any type T. And, if it is a JdbcTable, we'd like to be
-      // able to generate alternate accessors that return e.g. synthetic
-      // records {T0 f0; T1 f1; ...} and don't box every primitive value.
-      final PhysType physType =
-          PhysTypeImpl.of(
-              implementor.getTypeFactory(),
-              getRowType(),
-              format());
-      final Expression expression = getExpression(physType);
-      return implementor.result(physType, Blocks.toBlock(expression));
-    }
-  }
-
-  /** Relational expression that executes its children using an interpreter.
-   *
-   * <p>Although quite a few kinds of {@link RelNode} can be interpreted,
-   * this is only created by default for {@link FilterableTable}
-   * and {@link ProjectableFilterableTable}.
-   */
-  public static class EnumerableInterpreterRel extends SingleRel
-      implements EnumerableRel {
-    private final double factor;
-
-    /** Creates an EnumerableInterpreterRel. */
-    public EnumerableInterpreterRel(RelOptCluster cluster,
-        RelTraitSet traitSet, RelNode input, double factor) {
-      super(cluster, traitSet, input);
-      this.factor = factor;
-    }
-
-    @Override public RelOptCost computeSelfCost(RelOptPlanner planner) {
-      return super.computeSelfCost(planner).multiplyBy(factor);
-    }
-
-    @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
-      return new EnumerableInterpreterRel(getCluster(), traitSet, sole(inputs),
-          factor);
-    }
-
-    public Result implement(EnumerableRelImplementor implementor, Prefer pref) {
-      final JavaTypeFactory typeFactory = implementor.getTypeFactory();
-      final BlockBuilder builder = new BlockBuilder();
-      final PhysType physType =
-          PhysTypeImpl.of(typeFactory, getRowType(), JavaRowFormat.ARRAY);
-      final Expression interpreter_ = builder.append("interpreter",
-          Expressions.new_(Interpreter.class,
-              implementor.getRootExpression(),
-              implementor.stash(getChild(), RelNode.class)));
-      final Expression sliced_ =
-          getRowType().getFieldCount() == 1
-              ? Expressions.call(BuiltinMethod.SLICE0.method, interpreter_)
-              : interpreter_;
-      builder.add(sliced_);
-      return implementor.result(physType, builder.toBlock());
-    }
+  private EnumerableRules() {
   }
 
   public static final EnumerableProjectRule ENUMERABLE_PROJECT_RULE =
       new EnumerableProjectRule();
 
-  /**
-   * Rule to convert a {@link ProjectRel} to an
-   * {@link EnumerableProjectRel}.
-   */
-  private static class EnumerableProjectRule
-      extends ConverterRule {
-    private EnumerableProjectRule() {
-      super(
-          ProjectRel.class,
-          Convention.NONE,
-          EnumerableConvention.INSTANCE,
-          "EnumerableProjectRule");
-    }
-
-    public RelNode convert(RelNode rel) {
-      final ProjectRel project = (ProjectRel) rel;
-
-      if (B && RexMultisetUtil.containsMultiset(project.getProjects(), true)
-          || RexOver.containsOver(project.getProjects(), null)) {
-        return null;
-      }
-
-      return new EnumerableProjectRel(
-          rel.getCluster(),
-          rel.getTraitSet().replace(EnumerableConvention.INSTANCE),
-          convert(
-              project.getChild(),
-              project.getChild().getTraitSet()
-                  .replace(EnumerableConvention.INSTANCE)),
-          project.getProjects(),
-          project.getRowType(),
-          ProjectRelBase.Flags.BOXED);
-    }
-  }
-
-  /** Implementation of {@link org.eigenbase.rel.ProjectRel} in
-   * {@link EnumerableConvention enumerable calling convention}. */
-  public static class EnumerableProjectRel
-      extends ProjectRelBase
-      implements EnumerableRel {
-    public EnumerableProjectRel(
-        RelOptCluster cluster,
-        RelTraitSet traitSet,
-        RelNode child,
-        List<? extends RexNode> exps,
-        RelDataType rowType,
-        int flags) {
-      super(cluster, traitSet, child, exps, rowType, flags);
-      assert getConvention() instanceof EnumerableConvention;
-    }
-
-    public EnumerableProjectRel copy(RelTraitSet traitSet, RelNode input,
-        List<RexNode> exps, RelDataType rowType) {
-      return new EnumerableProjectRel(getCluster(), traitSet, input,
-          exps, rowType, flags);
-    }
-
-    public Result implement(EnumerableRelImplementor implementor, Prefer pref) {
-      // EnumerableCalcRel is always better
-      throw new UnsupportedOperationException();
-    }
-  }
-
   public static final EnumerableFilterRule ENUMERABLE_FILTER_RULE =
       new EnumerableFilterRule();
 
-  /**
-   * Rule to convert a {@link FilterRel} to an
-   * {@link EnumerableFilterRel}.
-   */
-  private static class EnumerableFilterRule
-      extends ConverterRule {
-    private EnumerableFilterRule() {
-      super(
-          FilterRel.class,
-          Convention.NONE,
-          EnumerableConvention.INSTANCE,
-          "EnumerableFilterRule");
-    }
-
-    public RelNode convert(RelNode rel) {
-      final FilterRel filter = (FilterRel) rel;
-
-      if (B && RexMultisetUtil.containsMultiset(filter.getCondition(), true)
-          || RexOver.containsOver(filter.getCondition())) {
-        return null;
-      }
-
-      return new EnumerableFilterRel(
-          rel.getCluster(),
-          rel.getTraitSet().replace(EnumerableConvention.INSTANCE),
-          convert(
-              filter.getChild(),
-              filter.getChild().getTraitSet()
-                  .replace(EnumerableConvention.INSTANCE)),
-          filter.getCondition());
-    }
-  }
-
-  /** Implementation of {@link org.eigenbase.rel.FilterRel} in
-   * {@link EnumerableConvention enumerable calling convention}. */
-  public static class EnumerableFilterRel
-      extends FilterRelBase
-      implements EnumerableRel {
-    public EnumerableFilterRel(
-        RelOptCluster cluster,
-        RelTraitSet traitSet,
-        RelNode child,
-        RexNode condition) {
-      super(cluster, traitSet, child, condition);
-      assert getConvention() instanceof EnumerableConvention;
-    }
-
-    public EnumerableFilterRel copy(RelTraitSet traitSet, RelNode input,
-        RexNode condition) {
-      return new EnumerableFilterRel(getCluster(), traitSet, input, condition);
-    }
-
-    public Result implement(EnumerableRelImplementor implementor, Prefer pref) {
-      // EnumerableCalcRel is always better
-      throw new UnsupportedOperationException();
-    }
-  }
-
   public static final EnumerableCalcRule ENUMERABLE_CALC_RULE =
       new EnumerableCalcRule();
 
-  /**
-   * Rule to convert a {@link CalcRel} to an
-   * {@link EnumerableCalcRel}.
-   */
-  private static class EnumerableCalcRule
-      extends ConverterRule {
-    private EnumerableCalcRule() {
-      super(
-          CalcRel.class,
-          Convention.NONE,
-          EnumerableConvention.INSTANCE,
-          "EnumerableCalcRule");
-    }
-
-    public RelNode convert(RelNode rel) {
-      final CalcRel calc = (CalcRel) rel;
-
-      // If there's a multiset, let FarragoMultisetSplitter work on it
-      // first.
-      final RexProgram program = calc.getProgram();
-      if (B && RexMultisetUtil.containsMultiset(program)
-          || program.containsAggs()) {
-        return null;
-      }
-
-      return new EnumerableCalcRel(
-          rel.getCluster(),
-          rel.getTraitSet().replace(EnumerableConvention.INSTANCE),
-          convert(
-              calc.getChild(),
-              calc.getChild().getTraitSet()
-                  .replace(EnumerableConvention.INSTANCE)),
-          calc.getRowType(),
-          program,
-          calc.getCollationList());
-    }
-  }
-
-  /** Implementation of {@link org.eigenbase.rel.CalcRel} in
-   * {@link EnumerableConvention enumerable calling convention}. */
-  public static class EnumerableCalcRel
-      extends CalcRelBase
-      implements EnumerableRel {
-    public EnumerableCalcRel(
-        RelOptCluster cluster,
-        RelTraitSet traitSet,
-        RelNode child,
-        RelDataType rowType,
-        RexProgram program,
-        List<RelCollation> collationList) {
-      super(cluster, traitSet, child, rowType, program, collationList);
-      assert getConvention() instanceof EnumerableConvention;
-      assert !program.containsAggs();
-    }
-
-    @Override public EnumerableCalcRel copy(RelTraitSet traitSet, RelNode child,
-        RexProgram program, List<RelCollation> collationList) {
-      // we do not need to copy program; it is immutable
-      return new EnumerableCalcRel(getCluster(), traitSet, child,
-          program.getOutputRowType(), program, collationList);
-    }
-
-    public Result implement(EnumerableRelImplementor implementor, Prefer pref) {
-      final JavaTypeFactory typeFactory = implementor.getTypeFactory();
-      final BlockBuilder builder = new BlockBuilder();
-      final EnumerableRel child = (EnumerableRel) getChild();
-
-      final Result result =
-          implementor.visitChild(this, 0, child, pref);
-
-      final PhysType physType =
-          PhysTypeImpl.of(
-              typeFactory, getRowType(), pref.prefer(result.format));
-
-      // final Enumerable<Employee> inputEnumerable = <<child impl>>;
-      // return new Enumerable<IntString>() {
-      //     Enumerator<IntString> enumerator() {
-      //         return new Enumerator<IntString>() {
-      //             public void reset() {
-      // ...
-      Type outputJavaType = physType.getJavaRowType();
-      final Type enumeratorType =
-          Types.of(
-              Enumerator.class, outputJavaType);
-      Type inputJavaType = result.physType.getJavaRowType();
-      ParameterExpression inputEnumerator =
-          Expressions.parameter(
-              Types.of(
-                  Enumerator.class, inputJavaType),
-              "inputEnumerator");
-      Expression input =
-          RexToLixTranslator.convert(
-              Expressions.call(
-                  inputEnumerator,
-                  BuiltinMethod.ENUMERATOR_CURRENT.method),
-              inputJavaType);
-
-      BlockStatement moveNextBody;
-      if (program.getCondition() == null) {
-        moveNextBody =
-            Blocks.toFunctionBlock(
-                Expressions.call(
-                    inputEnumerator,
-                    BuiltinMethod.ENUMERATOR_MOVE_NEXT.method));
-      } else {
-        final BlockBuilder builder2 = new BlockBuilder();
-        Expression condition =
-            RexToLixTranslator.translateCondition(
-                program,
-                typeFactory,
-                builder2,
-                new RexToLixTranslator.InputGetterImpl(
-                    Collections.singletonList(
-                        Pair.of(input, result.physType))));
-        builder2.add(
-            Expressions.ifThen(
-                condition,
-                Expressions.return_(
-                    null, Expressions.constant(true))));
-        moveNextBody =
-            Expressions.block(
-                Expressions.while_(
-                    Expressions.call(
-                        inputEnumerator,
-                        BuiltinMethod.ENUMERATOR_MOVE_NEXT.method),
-                    builder2.toBlock()),
-                Expressions.return_(
-                    null,
-                    Expressions.constant(false)));
-      }
-
-      final BlockBuilder builder3 = new BlockBuilder();
-      List<Expression> expressions =
-          RexToLixTranslator.translateProjects(
-              program,
-              typeFactory,
-              builder3,
-              physType,
-              new RexToLixTranslator.InputGetterImpl(
-                  Collections.singletonList(
-                      Pair.of(input, result.physType))));
-      builder3.add(
-          Expressions.return_(
-              null, physType.record(expressions)));
-      BlockStatement currentBody =
-          builder3.toBlock();
-
-      final Expression inputEnumerable =
-          builder.append(
-              "inputEnumerable", result.block, false);
-      final Expression body =
-          Expressions.new_(
-              enumeratorType,
-              NO_EXPRS,
-              Expressions.<MemberDeclaration>list(
-                  Expressions.fieldDecl(
-                      Modifier.PUBLIC
-                      | Modifier.FINAL,
-                      inputEnumerator,
-                      Expressions.call(
-                          inputEnumerable,
-                          BuiltinMethod.ENUMERABLE_ENUMERATOR.method)),
-                  EnumUtil.overridingMethodDecl(
-                      BuiltinMethod.ENUMERATOR_RESET.method,
-                      NO_PARAMS,
-                      Blocks.toFunctionBlock(
-                          Expressions.call(
-                              inputEnumerator,
-                              BuiltinMethod.ENUMERATOR_RESET.method))),
-                  EnumUtil.overridingMethodDecl(
-                      BuiltinMethod.ENUMERATOR_MOVE_NEXT.method,
-                      NO_PARAMS,
-                      moveNextBody),
-                  EnumUtil.overridingMethodDecl(
-                      BuiltinMethod.ENUMERATOR_CLOSE.method,
-                      NO_PARAMS,
-                      Blocks.toFunctionBlock(
-                          Expressions.call(
-                              inputEnumerator,
-                              BuiltinMethod.ENUMERATOR_CLOSE.method))),
-                  Expressions.methodDecl(
-                      Modifier.PUBLIC,
-                      BRIDGE_METHODS
-                          ? Object.class
-                          : outputJavaType,
-                      "current",
-                      NO_PARAMS,
-                      currentBody)));
-      builder.add(
-          Expressions.return_(
-              null,
-              Expressions.new_(
-                  BuiltinMethod.ABSTRACT_ENUMERABLE_CTOR.constructor,
-                  // TODO: generics
-                  //   Collections.singletonList(inputRowType),
-                  NO_EXPRS,
-                  ImmutableList.<MemberDeclaration>of(
-                      Expressions.methodDecl(
-                          Modifier.PUBLIC,
-                          enumeratorType,
-                          BuiltinMethod.ENUMERABLE_ENUMERATOR.method.getName(),
-                          NO_PARAMS,
-                          Blocks.toFunctionBlock(body))))));
-      return implementor.result(physType, builder.toBlock());
-    }
-
-    public RexProgram getProgram() {
-      return program;
-    }
-  }
-
   public static final EnumerableAggregateRule ENUMERABLE_AGGREGATE_RULE =
       new EnumerableAggregateRule();
 
-  /**
-   * Rule to convert an {@link org.eigenbase.rel.AggregateRel} to an
-   * {@link net.hydromatic.optiq.rules.java.JavaRules.EnumerableAggregateRel}.
-   */
-  private static class EnumerableAggregateRule
-      extends ConverterRule {
-    private EnumerableAggregateRule() {
-      super(
-          AggregateRel.class,
-          Convention.NONE,
-          EnumerableConvention.INSTANCE,
-          "EnumerableAggregateRule");
-    }
-
-    public RelNode convert(RelNode rel) {
-      final AggregateRel agg = (AggregateRel) rel;
-      final RelTraitSet traitSet =
-          agg.getTraitSet().replace(EnumerableConvention.INSTANCE);
-      try {
-        return new EnumerableAggregateRel(
-            rel.getCluster(),
-            traitSet,
-            convert(agg.getChild(), traitSet),
-            agg.getGroupSet(),
-            agg.getAggCallList());
-      } catch (InvalidRelException e) {
-        LOGGER.fine(e.toString());
-        return null;
-      }
-    }
-  }
-
-  /** Implementation of {@link org.eigenbase.rel.AggregateRel} in
-   * {@link EnumerableConvention enumerable calling convention}. */
-  public static class EnumerableAggregateRel
-      extends AggregateRelBase
-      implements EnumerableRel {
-    private static final List<Aggregation> SUPPORTED_AGGREGATIONS =
-        ImmutableList.<Aggregation>of(
-            SqlStdOperatorTable.COUNT,
-            SqlStdOperatorTable.MIN,
-            SqlStdOperatorTable.MAX,
-            SqlStdOperatorTable.SUM);
-
-    public EnumerableAggregateRel(
-        RelOptCluster cluster,
-        RelTraitSet traitSet,
-        RelNode child,
-        BitSet groupSet,
-        List<AggregateCall> aggCalls)
-        throws InvalidRelException {
-      super(cluster, traitSet, child, groupSet, aggCalls);
-      assert getConvention() instanceof EnumerableConvention;
-
-      for (AggregateCall aggCall : aggCalls) {
-        if (aggCall.isDistinct()) {
-          throw new InvalidRelException(
-              "distinct aggregation not supported");
-        }
-        AggImplementor implementor2 =
-            RexImpTable.INSTANCE.get(aggCall.getAggregation(), false);
-        if (implementor2 == null) {
-          throw new InvalidRelException(
-              "aggregation " + aggCall.getAggregation() + " not supported");
-        }
-      }
-    }
-
-    @Override public EnumerableAggregateRel copy(RelTraitSet traitSet,
-        RelNode input, BitSet groupSet, List<AggregateCall> aggCalls) {
-      try {
-        return new EnumerableAggregateRel(getCluster(), traitSet, input,
-            groupSet, aggCalls);
-      } catch (InvalidRelException e) {
-        // Semantic error not possible. Must be a bug. Convert to
-        // internal error.
-        throw new AssertionError(e);
-      }
-    }
-
-    public Result implement(EnumerableRelImplementor implementor, Prefer pref) {
-      final JavaTypeFactory typeFactory = implementor.getTypeFactory();
-      final BlockBuilder builder = new BlockBuilder();
-      final EnumerableRel child = (EnumerableRel) getChild();
-      final Result result = implementor.visitChild(this, 0, child, pref);
-      Expression childExp =
-          builder.append(
-              "child",
-              result.block);
-      final RelDataType inputRowType = getChild().getRowType();
-
-      final PhysType physType =
-          PhysTypeImpl.of(
-              typeFactory, getRowType(), pref.preferCustom());
-
-      // final Enumerable<Employee> child = <<child impl>>;
-      // Function1<Employee, Integer> keySelector =
-      //     new Function1<Employee, Integer>() {
-      //         public Integer apply(Employee a0) {
-      //             return a0.deptno;
-      //         }
-      //     };
-      // Function1<Employee, Object[]> accumulatorInitializer =
-      //     new Function1<Employee, Object[]>() {
-      //         public Object[] apply(Employee a0) {
-      //             return new Object[] {0, 0};
-      //         }
-      //     };
-      // Function2<Object[], Employee, Object[]> accumulatorAdder =
-      //     new Function2<Object[], Employee, Object[]>() {
-      //         public Object[] apply(Object[] a1, Employee a0) {
-      //              a1[0] = ((Integer) a1[0]) + 1;
-      //              a1[1] = ((Integer) a1[1]) + a0.salary;
-      //             return a1;
-      //         }
-      //     };
-      // Function2<Integer, Object[], Object[]> resultSelector =
-      //     new Function2<Integer, Object[], Object[]>() {
-      //         public Object[] apply(Integer a0, Object[] a1) {
-      //             return new Object[] { a0, a1[0], a1[1] };
-      //         }
-      //     };
-      // return childEnumerable
-      //     .groupBy(
-      //        keySelector, accumulatorInitializer, accumulatorAdder,
-      //        resultSelector);
-      //
-      // or, if key has 0 columns,
-      //
-      // return childEnumerable
-      //     .aggregate(
-      //       accumulatorInitializer.apply(),
-      //       accumulatorAdder,
-      //       resultSelector);
-      //
-      // with a slightly different resultSelector; or if there are no aggregate
-      // functions
-      //
-      // final Enumerable<Employee> child = <<child impl>>;
-      // Function1<Employee, Integer> keySelector =
-      //     new Function1<Employee, Integer>() {
-      //         public Integer apply(Employee a0) {
-      //             return a0.deptno;
-      //         }
-      //     };
-      // EqualityComparer<Employee> equalityComparer =
-      //     new EqualityComparer<Employee>() {
-      //         boolean equal(Employee a0, Employee a1) {
-      //             return a0.deptno;
-      //         }
-      //     };
-      // return child
-      //     .distinct(equalityComparer);
-
-      final PhysType inputPhysType = result.physType;
-
-      ParameterExpression parameter =
-          Expressions.parameter(inputPhysType.getJavaRowType(), "a0");
-
-      final PhysType keyPhysType =
-          inputPhysType.project(
-              BitSets.toList(groupSet), JavaRowFormat.LIST);
-      final int keyArity = groupSet.cardinality();
-      final Expression keySelector =
-          builder.append(
-              "keySelector",
-              inputPhysType.generateSelector(
-                  parameter,
-                  BitSets.toList(groupSet),
-                  keyPhysType.getFormat()));
-
-      final List<AggImpState> aggs =
-          new ArrayList<AggImpState>(aggCalls.size());
-
-      for (int i = 0; i < aggCalls.size(); i++) {
-        AggregateCall call = aggCalls.get(i);
-        aggs.add(new AggImpState(i, call, false));
-      }
-
-      // Function0<Object[]> accumulatorInitializer =
-      //     new Function0<Object[]>() {
-      //         public Object[] apply() {
-      //             return new Object[] {0, 0};
-      //         }
-      //     };
-      final List<Expression> initExpressions =
-          new ArrayList<Expression>();
-      final BlockBuilder initBlock = new BlockBuilder();
-
-      final List<Type> aggStateTypes = new ArrayList<Type>();
-      for (final AggImpState agg : aggs) {
-        agg.context =
-            new AggContext() {
-              public Aggregation aggregation() {
-                return agg.call.getAggregation();
-              }
-
-              public RelDataType returnRelType() {
-                return agg.call.type;
-              }
-
-              public Type returnType() {
-                return EnumUtil.javaClass(typeFactory, returnRelType());
-              }
-
-              public List<? extends RelDataType> parameterRelTypes() {
-                return EnumUtil.fieldRowTypes(inputRowType, null,
-                    agg.call.getArgList());
-              }
-
-              public List<? extends Type> parameterTypes() {
-                return EnumUtil.fieldTypes(typeFactory,
-                    parameterRelTypes());
-              }
-            };
-        List<Type> state =
-            agg.implementor.getStateType(agg.context);
-
-        if (state.isEmpty()) {
-          continue;
-        }
-
-        aggStateTypes.addAll(state);
-
-        final List<Expression> decls =
-            new ArrayList<Expression>(state.size());
-        for (int i = 0; i < state.size(); i++) {
-          String aggName = "a" + agg.aggIdx;
-          if (OptiqPrepareImpl.DEBUG) {
-            aggName = Util.toJavaId(agg.call.getAggregation().getName(), 0)
-                .substring("ID$0$".length()) + aggName;
-          }
-          Type type = state.get(i);
-          ParameterExpression pe =
-              Expressions.parameter(type,
-                  initBlock.newName(aggName + "s" + i));
-          initBlock.add(Expressions.declare(0, pe, null));
-          decls.add(pe);
-        }
-        agg.state = decls;
-        initExpressions.addAll(decls);
-        agg.implementor.implementReset(agg.context,
-            new AggResultContextImpl(initBlock, decls));
-      }
-
-      final PhysType accPhysType =
-          PhysTypeImpl.of(
-              typeFactory,
-              typeFactory.createSyntheticType(aggStateTypes));
-
-      initBlock.add(accPhysType.record(initExpressions));
-
-      final Expression accumulatorInitializer =
-          builder.append(
-              "accumulatorInitializer",
-              Expressions.lambda(
-                  Function0.class,
-                  initBlock.toBlock()));
-
-      // Function2<Object[], Employee, Object[]> accumulatorAdder =
-      //     new Function2<Object[], Employee, Object[]>() {
-      //         public Object[] apply(Object[] acc, Employee in) {
-      //              acc[0] = ((Integer) acc[0]) + 1;
-      //              acc[1] = ((Integer) acc[1]) + in.salary;
-      //             return acc;
-      //         }
-      //     };
-      final BlockBuilder builder2 = new BlockBuilder();
-      final ParameterExpression inParameter =
-          Expressions.parameter(inputPhysType.getJavaRowType(), "in");
-      final ParameterExpression acc_ =
-          Expressions.parameter(accPhysType.getJavaRowType(), "acc");
-      for (int i = 0, stateOffset = 0; i < aggs.size(); i++) {
-        final AggImpState agg = aggs.get(i);
-
-        int stateSize = agg.state.size();
-        List<Expression> accumulator =
-            new ArrayList<Expression>(stateSize);
-        for (int j = 0; j < stateSize; j++) {
-          accumulator.add(accPhysType.fieldReference(
-              acc_, j + stateOffset));
-        }
-        agg.state = accumulator;
-
-        stateOffset += stateSize;
-
-        AggAddContext addContext =
-            new AggAddContextImpl(builder2, accumulator) {
-              public List<RexNode> rexArguments() {
-                List<RelDataTypeField> inputTypes =
-                    inputPhysType.getRowType().getFieldList();
-                List<RexNode> args = new ArrayList<RexNode>();
-                for (Integer index : agg.call.getArgList()) {
-                  args.add(new RexInputRef(index,
-                      inputTypes.get(index).getType()));
-                }
-                return args;
-              }
-
-              public RexToLixTranslator rowTranslator() {
-                return RexToLixTranslator.forAggregation(typeFactory,
-                    currentBlock(), new RexToLixTranslator.InputGetterImpl(
-                        Collections.singletonList(Pair.of(
-                            (Expression) inParameter, inputPhysType))))
-                    .setNullable(currentNullables());
-              }
-            };
-
-        agg.implementor.implementAdd(agg.context, addContext);
-      }
-      builder2.add(acc_);
-      final Expression accumulatorAdder =
-          builder.append(
-              "accumulatorAdder",
-              Expressions.lambda(
-                  Function2.class,
-                  builder2.toBlock(),
-                  acc_,
-                  inParameter));
-
-      // Function2<Integer, Object[], Object[]> resultSelector =
-      //     new Function2<Integer, Object[], Object[]>() {
-      //         public Object[] apply(Integer key, Object[] acc) {
-      //             return new Object[] { key, acc[0], acc[1] };
-      //         }
-      //     };
-      final BlockBuilder resultBlock = new BlockBuilder();
-      final List<Expression> results = Expressions.list();
-      final ParameterExpression key_;
-      if (keyArity == 0) {
-        key_ = null;
-      } else {
-        final Type keyType = keyPhysType.getJavaRowType();
-        key_ = Expressions.parameter(keyType, "key");
-        for (int j = 0; j < keyArity; j++) {
-          results.add(
-              keyPhysType.fieldReference(key_, j));
-        }
-      }
-      for (final AggImpState agg : aggs) {
-        results.add(
-            agg.implementor.implementResult(agg.context,
-                new AggResultContextImpl(resultBlock, agg.state)));
-      }
-      resultBlock.add(physType.record(results));
-      if (keyArity == 0) {
-        final Expression resultSelector =
-            builder.append(
-                "resultSelector",
-                Expressions.lambda(
-                    Function1.class,
-                    resultBlock.toBlock(),
-                    acc_));
-        builder.add(
-            Expressions.return_(
-                null,
-                Expressions.call(
-                    BuiltinMethod.SINGLETON_ENUMERABLE.method,
-                    Expressions.call(
-                        childExp,
-                        BuiltinMethod.AGGREGATE.method,
-                        Expressions.call(accumulatorInitializer, "apply"),
-                        accumulatorAdder,
-                        resultSelector))));
-      } else if (aggCalls.isEmpty()
-          && groupSet.equals(
-              BitSets.range(child.getRowType().getFieldCount()))) {
-        builder.add(
-            Expressions.return_(
-                null,
-                Expressions.call(
-                    inputPhysType.convertTo(childExp, physType),
-                    BuiltinMethod.DISTINCT.method,
-                    Expressions.<Expression>list()
-                        .appendIfNotNull(physType.comparer()))));
-      } else {
-        final Expression resultSelector =
-            builder.append(
-                "resultSelector",
-                Expressions.lambda(
-                    Function2.class,
-                    resultBlock.toBlock(),
-                    key_,
-                    acc_));
-        builder.add(
-            Expressions.return_(
-                null,
-                Expressions.call(
-                    childExp,
-                    BuiltinMethod.GROUP_BY2.method,
-                    Expressions
-                        .list(
-                            keySelector,
-                            accumulatorInitializer,
-                            accumulatorAdder,
-                            resultSelector)
-                        .appendIfNotNull(
-                            keyPhysType.comparer()))));
-      }
-      return implementor.result(physType, builder.toBlock());
-    }
-  }
-
   public static final EnumerableSortRule ENUMERABLE_SORT_RULE =
       new EnumerableSortRule();
 
-  /**
-   * Rule to convert an {@link org.eigenbase.rel.SortRel} to an
-   * {@link net.hydromatic.optiq.rules.java.JavaRules.EnumerableSortRel}.
-   */
-  private static class EnumerableSortRule
-      extends ConverterRule {
-    private EnumerableSortRule() {
-      super(
-          SortRel.class,
-          Convention.NONE,
-          EnumerableConvention.INSTANCE,
-          "EnumerableSortRule");
-    }
-
-    public RelNode convert(RelNode rel) {
-      final SortRel sort = (SortRel) rel;
-      if (sort.offset != null || sort.fetch != null) {
-        return null;
-      }
-      final RelTraitSet traitSet =
-          sort.getTraitSet().replace(EnumerableConvention.INSTANCE);
-      final RelNode input = sort.getChild();
-      return new EnumerableSortRel(
-          rel.getCluster(),
-          traitSet,
-          convert(
-              input,
-              input.getTraitSet().replace(EnumerableConvention.INSTANCE)),
-          sort.getCollation(),
-          null,
-          null);
-    }
-  }
-
-  /** Implementation of {@link org.eigenbase.rel.SortRel} in
-   * {@link EnumerableConvention enumerable calling convention}. */
-  public static class EnumerableSortRel
-      extends SortRel
-      implements EnumerableRel {
-    public EnumerableSortRel(RelOptCluster cluster, RelTraitSet traitSet,
-        RelNode child, RelCollation collation, RexNode offset, RexNode fetch) {
-      super(cluster, traitSet, child, collation, offset, fetch);
-      assert getConvention() instanceof EnumerableConvention;
-      assert getConvention() == child.getConvention();
-    }
-
-    @Override
-    public EnumerableSortRel copy(
-        RelTraitSet traitSet,
-        RelNode newInput,
-        RelCollation newCollation,
-        RexNode offset,
-        RexNode fetch) {
-      return new EnumerableSortRel(
-          getCluster(),
-          traitSet,
-          newInput,
-          newCollation,
-          offset,
-          fetch);
-    }
-
-    public Result implement(EnumerableRelImplementor implementor, Prefer pref) {
-      final BlockBuilder builder = new BlockBuilder();
-      final EnumerableRel child = (EnumerableRel) getChild();
-      final Result result = implementor.visitChild(this, 0, child, pref);
-      final PhysType physType =
-          PhysTypeImpl.of(
-              implementor.getTypeFactory(),
-              getRowType(),
-              result.format);
-      Expression childExp =
-          builder.append(
-              "child", result.block);
-
-      PhysType inputPhysType = result.physType;
-      final Pair<Expression, Expression> pair =
-          inputPhysType.generateCollationKey(
-              collation.getFieldCollations());
-
-      builder.add(
-          Expressions.return_(
-              null,
-              Expressions.call(
-                  childExp,
-                  BuiltinMethod.ORDER_BY.method,
-                  Expressions.list(
-                      builder.append("keySelector", pair.left))
-                  .appendIfNotNull(builder.appendIfNotNull("comparator",
-                      pair.right)))));
-      return implementor.result(physType, builder.toBlock());
-    }
-  }
-
   public static final EnumerableLimitRule ENUMERABLE_LIMIT_RULE =
       new EnumerableLimitRule();
 
-  /**
-   * Rule to convert an {@link org.eigenbase.rel.SortRel} that has
-   * {@code offset} or {@code fetch} set to an
-   * {@link net.hydromatic.optiq.rules.java.JavaRules.EnumerableLimitRel}
-   * on top of a "pure" {@code SortRel} that has no offset or fetch.
-   */
-  private static class EnumerableLimitRule
-      extends RelOptRule {
-    private EnumerableLimitRule() {
-      super(
-          operand(SortRel.class, any()),
-          "EnumerableLimitRule");
-    }
-
-    @Override
-    public void onMatch(RelOptRuleCall call) {
-      final SortRel sort = call.rel(0);
-      if (sort.offset == null && sort.fetch == null) {
-        return;
-      }
-      final RelTraitSet traitSet =
-          sort.getTraitSet().replace(EnumerableConvention.INSTANCE);
-      RelNode input = sort.getChild();
-      if (!sort.getCollation().getFieldCollations().isEmpty()) {
-        // Create a sort with the same sort key, but no offset or fetch.
-        input = sort.copy(
-            sort.getTraitSet(),
-            input,
-            sort.getCollation(),
-            null,
-            null);
-      }
-      RelNode x = convert(
-          input,
-          input.getTraitSet().replace(EnumerableConvention.INSTANCE));
-      call.transformTo(
-          new EnumerableLimitRel(
-              sort.getCluster(),
-              traitSet,
-              x,
-              sort.offset,
-              sort.fetch));
-    }
-  }
-
-  /** Relational expression that applies a limit and/or offset to its input. */
-  public static class EnumerableLimitRel
-      extends SingleRel
-      implements EnumerableRel {
-    private final RexNode offset;
-    private final RexNode fetch;
-
-    public EnumerableLimitRel(
-        RelOptCluster cluster,
-        RelTraitSet traitSet,
-        RelNode child,
-        RexNode offset,
-        RexNode fetch) {
-      super(cluster, traitSet, child);
-      this.offset = offset;
-      this.fetch = fetch;
-      assert getConvention() instanceof EnumerableConvention;
-      assert getConvention() == child.getConvention();
-    }
-
-    @Override
-    public EnumerableLimitRel copy(
-        RelTraitSet traitSet,
-        List<RelNode> newInputs) {
-      return new EnumerableLimitRel(
-          getCluster(),
-          traitSet,
-          sole(newInputs),
-          offset,
-          fetch);
-    }
-
-    @Override
-    public RelWriter explainTerms(RelWriter pw) {
-      return super.explainTerms(pw)
-          .itemIf("offset", offset, offset != null)
-          .itemIf("fetch", fetch, fetch != null);
-    }
-
-    public Result implement(EnumerableRelImplementor implementor, Prefer pref) {
-      final BlockBuilder builder = new BlockBuilder();
-      final EnumerableRel child = (EnumerableRel) getChild();
-      final Result result = implementor.visitChild(this, 0, child, pref);
-      final PhysType physType =
-          PhysTypeImpl.of(
-              implementor.getTypeFactory(),
-              getRowType(),
-              result.format);
-      Expression childExp =
-          builder.append(
-              "child", result.block);
-
-      Expression v = childExp;
-      if (offset != null) {
-        v = builder.append(
-            "offset",
-            Expressions.call(
-                v,
-                BuiltinMethod.SKIP.method,
-                Expressions.constant(RexLiteral.intValue(offset))));
-      }
-      if (fetch != null) {
-        v = builder.append(
-            "fetch",
-            Expressions.call(
-                v,
-                BuiltinMethod.TAKE.method,
-                Expressions.constant(RexLiteral.intValue(fetch))));
-      }
-
-      builder.add(
-          Expressions.return_(
-              null,
-              v));
-      return implementor.result(physType, builder.toBlock());
-    }
-  }
-
   public static final EnumerableUnionRule ENUMERABLE_UNION_RULE =
       new EnumerableUnionRule();
 
-  /**
-   * Rule to convert an {@link org.eigenbase.rel.UnionRel} to an
-   * {@link net.hydromatic.optiq.rules.java.JavaRules.EnumerableUnionRel}.
-   */
-  private static class EnumerableUnionRule
-      extends ConverterRule {
-    private EnumerableUnionRule() {
-      super(
-          UnionRel.class,
-          Convention.NONE,
-          EnumerableConvention.INSTANCE,
-          "EnumerableUnionRule");
-    }
-
-    public RelNode convert(RelNode rel) {
-      final UnionRel union = (UnionRel) rel;
-      final EnumerableConvention out = EnumerableConvention.INSTANCE;
-      final RelTraitSet traitSet = union.getTraitSet().replace(out);
-      return new EnumerableUnionRel(rel.getCluster(), traitSet,
-          convertList(union.getInputs(), out), union.all);
-    }
-  }
-
-  /** Implementation of {@link org.eigenbase.rel.UnionRel} in
-   * {@link EnumerableConvention enumerable calling convention}. */
-  public static class EnumerableUnionRel
-      extends UnionRelBase
-      implements EnumerableRel {
-    public EnumerableUnionRel(
-        RelOptCluster cluster,
-        RelTraitSet traitSet,
-        List<RelNode> inputs,
-        boolean all) {
-      super(cluster, traitSet, inputs, all);
-    }
-
-    public EnumerableUnionRel copy(
-        RelTraitSet traitSet, List<RelNode> inputs, boolean all) {
-      return new EnumerableUnionRel(getCluster(), traitSet, inputs, all);
-    }
-
-    public Result implement(EnumerableRelImplementor implementor, Prefer pref) {
-      final BlockBuilder builder = new BlockBuilder();
-      Expression unionExp = null;
-      for (Ord<RelNode> ord : Ord.zip(inputs)) {
-        EnumerableRel input = (EnumerableRel) ord.e;
-        final Result result = implementor.visitChild(this, ord.i, input, pref);
-        Expression childExp =
-            builder.append(
-                "child" + ord.i,
-                result.block);
-
-        if (unionExp == null) {
-          unionExp = childExp;
-        } else {
-          unionExp =
-              Expressions.call(
-                  unionExp,
-                  all
-                      ? BuiltinMethod.CONCAT.method
-                      : BuiltinMethod.UNION.method,
-                  childExp);
-        }
-
-        // Once the first input has chosen its format, ask for the same for
-        // other inputs.
-        pref = pref.of(result.format);
-      }
-
-      builder.add(unionExp);
-      final PhysType physType =
-          PhysTypeImpl.of(
-              implementor.getTypeFactory(),
-              getRowType(),
-              pref.prefer(JavaRowFormat.CUSTOM));
-      return implementor.result(physType, builder.toBlock());
-    }
-  }
-
   public static final EnumerableIntersectRule ENUMERABLE_INTERSECT_RULE =
       new EnumerableIntersectRule();
 
-  /**
-   * Rule to convert an {@link org.eigenbase.rel.IntersectRel} to an
-   * {@link net.hydromatic.optiq.rules.java.JavaRules.EnumerableIntersectRel}.
-   */
-  private static class EnumerableIntersectRule
-      extends ConverterRule {
-    private EnumerableIntersectRule() {
-      super(
-          IntersectRel.class,
-          Convention.NONE,
-          EnumerableConvention.INSTANCE,
-          "EnumerableIntersectRule");
-    }
-
-    public RelNode convert(RelNode rel) {
-      final IntersectRel intersect = (IntersectRel) rel;
-      if (intersect.all) {
-        return null; // INTERSECT ALL not implemented
-      }
-      final EnumerableConvention out = EnumerableConvention.INSTANCE;
-      final RelTraitSet traitSet = intersect.getTraitSet().replace(out);
-      return new EnumerableIntersectRel(rel.getCluster(), traitSet,
-          convertList(intersect.getInputs(), out), intersect.all);
-    }
-  }
-
-  /** Implementation of {@link org.eigenbase.rel.IntersectRel} in
-   * {@link EnumerableConvention enumerable calling convention}. */
-  public static class EnumerableIntersectRel
-      extends IntersectRelBase
-      implements EnumerableRel {
-    public EnumerableIntersectRel(
-        RelOptCluster cluster,
-        RelTraitSet traitSet,
-        List<RelNode> inputs,
-        boolean all) {
-      super(cluster, traitSet, inputs, all);
-      assert !all;
-    }
-
-    public EnumerableIntersectRel copy(
-        RelTraitSet traitSet, List<RelNode> inputs, boolean all) {
-      return new EnumerableIntersectRel(
-          getCluster(),
-          traitSet,
-          inputs,
-          all);
-    }
-
-    public Result implement(EnumerableRelImplementor implementor, Prefer pref) {
-      final BlockBuilder builder = new BlockBuilder();
-      Expression intersectExp = null;
-      for (Ord<RelNode> ord : Ord.zip(inputs)) {
-        EnumerableRel input = (EnumerableRel) ord.e;
-        final Result result = implementor.visitChild(this, ord.i, input, pref);
-        Expression childExp =
-            builder.append(
-                "child" + ord.i,
-                result.block);
-
-        if (intersectExp == null) {
-          intersectExp = childExp;
-        } else {
-          intersectExp =
-              Expressions.call(
-                  intersectExp,
-                  all
-                      ? BuiltinMethod.CONCAT.method
-                      : BuiltinMethod.INTERSECT.method,
-                  childExp);
-        }
-
-        // Once the first input has chosen its format, ask for the same for
-        // other inputs.
-        pref = pref.of(result.format);
-      }
-
-      builder.add(intersectExp);
-      final PhysType physType =
-          PhysTypeImpl.of(
-              implementor.getTypeFactory(),
-              getRowType(),
-              pref.prefer(JavaRowFormat.CUSTOM));
-      return implementor.result(physType, builder.toBlock());
-    }
-  }
-
   public static final EnumerableMinusRule ENUMERABLE_MINUS_RULE =
       new EnumerableMinusRule();
 
-  /**
-   * Rule to convert an {@link org.eigenbase.rel.MinusRel} to an
-   * {@link net.hydromatic.optiq.rules.java.JavaRules.EnumerableMinusRel}.
-   */
-  private static class EnumerableMinusRule
-      extends ConverterRule {
-    private EnumerableMinusRule() {
-      super(
-          MinusRel.class,
-          Convention.NONE,
-          EnumerableConvention.INSTANCE,
-          "EnumerableMinusRule");
-    }
-
-    public RelNode convert(RelNode rel) {
-      final MinusRel minus = (MinusRel) rel;
-      if (minus.all) {
-        return null; // EXCEPT ALL not implemented
-      }
-      final EnumerableConvention out = EnumerableConvention.INSTANCE;
-      final RelTraitSet traitSet =
-          rel.getTraitSet().replace(
-              EnumerableConvention.INSTANCE);
-      return new EnumerableMinusRel(rel.getCluster(), traitSet,
-          convertList(minus.getInputs(), out), minus.all);
-    }
-  }
-
-  /** Implementation of {@link org.eigenbase.rel.MinusRel} in
-   * {@link EnumerableConvention enumerable calling convention}. */
-  public static class EnumerableMinusRel
-      extends MinusRelBase
-      implements EnumerableRel {
-    public EnumerableMinusRel(
-        RelOptCluster cluster,
-        RelTraitSet traitSet,
-        List<RelNode> inputs,
-        boolean all) {
-      super(cluster, traitSet, inputs, all);
-      assert !all;
-    }
-
-    public EnumerableMinusRel copy(
-        RelTraitSet traitSet, List<RelNode> inputs, boolean all) {
-      return new EnumerableMinusRel(
-          getCluster(),
-          traitSet,
-          inputs,
-          all);
-    }
-
-    public Result implement(EnumerableRelImplementor implementor, Prefer pref) {
-      final BlockBuilder builder = new BlockBuilder();
-      Expression minusExp = null;
-      for (Ord<RelNode> ord : Ord.zip(inputs)) {
-        EnumerableRel input = (EnumerableRel) ord.e;
-        final Result result = implementor.visitChild(this, ord.i, input, pref);
-        Expression childExp =
-            builder.append(
-                "child" + ord.i,
-                result.block);
-
-        if (minusExp == null) {
-          minusExp = childExp;
-        } else {
-          minusExp =
-              Expressions.call(
-                  minusExp,
-                  BuiltinMethod.EXCEPT.method,
-                  childExp);
-        }
-
-        // Once the first input has chosen its format, ask for the same for
-        // other inputs.
-        pref = pref.of(result.format);
-      }
-
-      builder.add(minusExp);
-      final PhysType physType =
-          PhysTypeImpl.of(
-              implementor.getTypeFactory(),
-              getRowType(),
-              pref.prefer(JavaRowFormat.CUSTOM));
-      return implementor.result(physType, builder.toBlock());
-    }
-  }
-
-  public static final EnumerableTableModificationRule
-  ENUMERABLE_TABLE_MODIFICATION_RULE =
-      new EnumerableTableModificationRule();
-
-  public static class EnumerableTableModificationRule extends ConverterRule {
-    private EnumerableTableModificationRule() {
-      super(
-          TableModificationRel.class,
-          Convention.NONE,
-          EnumerableConvention.INSTANCE,
-          "EnumerableTableModificationRule");
-    }
-
-    @Override
-    public RelNode convert(RelNode rel) {
-      final TableModificationRel modify =
-          (TableModificationRel) rel;
-      final ModifiableTable modifiableTable =
-          modify.getTable().unwrap(ModifiableTable.class);
-      if (modifiableTable == null) {
-        return null;
-      }
-      final RelTraitSet traitSet =
-          modify.getTraitSet().replace(EnumerableConvention.INSTANCE);
-      return new EnumerableTableModificationRel(
-          modify.getCluster(), traitSet,
-          modify.getTable(),
-          modify.getCatalogReader(),
-          convert(modify.getChild(), traitSet),
-          modify.getOperation(),
-          modify.getUpdateColumnList(),
-          modify.isFlattened());
-    }
-  }
-
-  /** Implementation of {@link org.eigenbase.rel.TableModificationRel} in
-   * {@link EnumerableConvention enumerable calling convention}. */
-  public static class EnumerableTableModificationRel
-      extends TableModificationRelBase
-      implements EnumerableRel {
-    public EnumerableTableModificationRel(
-        RelOptCluster cluster,
-        RelTraitSet traits,
-        RelOptTable table,
-        Prepare.CatalogReader catalogReader,
-        RelNode child,
-        Operation operation,
-        List<String> updateColumnList,
-        boolean flattened) {
-      super(
-          cluster,
-          traits,
-          table,
-          catalogReader,
-          child,
-          operation,
-          updateColumnList,
-          flattened);
-      assert child.getConvention() instanceof EnumerableConvention;
-      assert getConvention() instanceof EnumerableConvention;
-      final ModifiableTable modifiableTable =
-          table.unwrap(ModifiableTable.class);
-      if (modifiableTable == null) {
-        throw new AssertionError(); // TODO: user error in validator
-      }
-    }
-
-    @Override
-    public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
-      return new EnumerableTableModificationRel(
-          getCluster(),
-          traitSet,
-          getTable(),
-          getCatalogReader(),
-          sole(inputs),
-          getOperation(),
-          getUpdateColumnList(),
-          isFlattened());
-    }
-
-    public Result implement(EnumerableRelImplementor implementor, Prefer pref) {
-      final BlockBuilder builder = new BlockBuilder();
-      final Result result = implementor.visitChild(
-          this, 0, (EnumerableRel) getChild(), pref);
-      Expression childExp =
-          builder.append(
-              "child", result.block);
-      final ParameterExpression collectionParameter =
-          Expressions.parameter(Collection.class,
-              builder.newName("collection"));
-      final Expression expression = table.getExpression(ModifiableTable.class);
-      assert expression != null; // TODO: user error in validator
-      assert ModifiableTable.class.isAssignableFrom(
-          Types.toClass(expression.getType())) : expression.getType();
-      builder.add(
-          Expressions.declare(
-              Modifier.FINAL,
-              collectionParameter,
-              Expressions.call(
-                  expression,
-                  BuiltinMethod.MODIFIABLE_TABLE_GET_MODIFIABLE_COLLECTION
-                      .method)));
-      final Expression countParameter =
-          builder.append(
-              "count",
-              Expressions.call(collectionParameter, "size"),
-              false);
-      Expression convertedChildExp;
-      if (!getChild().getRowType().equals(getRowType())) {
-        final JavaTypeFactory typeFactory =
-            (JavaTypeFactory) getCluster().getTypeFactory();
-        PhysType physType =
-            PhysTypeImpl.of(typeFactory, table.getRowType(),
-                JavaRowFormat.CUSTOM);
-        List<Expression> expressionList = new ArrayList<Expression>();
-        final PhysType childPhysType = result.physType;
-        final ParameterExpression o_ =
-            Expressions.parameter(childPhysType.getJavaRowType(), "o");
-        final int fieldCount =
-            childPhysType.getRowType().getFieldCount();
-        for (int i = 0; i < fieldCount; i++) {
-          expressionList.add(childPhysType.fieldReference(o_, i,
-              physType.getJavaFieldType(i)));
-        }
-        convertedChildExp =
-            builder.append(
-                "convertedChild",
-                Expressions.call(
-                    childExp,
-                    BuiltinMethod.SELECT.method,
-                    Expressions.lambda(
-                        physType.record(expressionList), o_)));
-      } else {
-        convertedChildExp = childExp;
-      }
-      builder.add(
-          Expressions.statement(
-              Expressions.call(
-                  convertedChildExp, "into", collectionParameter)));
-      builder.add(
-          Expressions.return_(
-              null,
-              Expressions.call(
-                  BuiltinMethod.SINGLETON_ENUMERABLE.method,
-                  Expressions.convert_(
-                      Expressions.subtract(
-                          Expressions.call(
-                              collectionParameter, "size"),
-                          countParameter),
-                      long.class))));
-      final PhysType physType =
-          PhysTypeImpl.of(
-              implementor.getTypeFactory(),
-              getRowType(),
-              pref == Prefer.ARRAY
-                  ? JavaRowFormat.ARRAY : JavaRowFormat.SCALAR);
-      return implementor.result(physType, builder.toBlock());
-    }
-  }
+  public static final EnumerableTableModifyRule
+  ENUMERABLE_TABLE_MODIFICATION_RULE = new EnumerableTableModifyRule();
 
   public static final EnumerableValuesRule ENUMERABLE_VALUES_RULE =
       new EnumerableValuesRule();
 
-  public static class EnumerableValuesRule extends ConverterRule {
-    private EnumerableValuesRule() {
-      super(
-          ValuesRel.class,
-          Convention.NONE,
-          EnumerableConvention.INSTANCE,
-          "EnumerableValuesRule");
-    }
-
-    @Override
-    public RelNode convert(RelNode rel) {
-      ValuesRel valuesRel = (ValuesRel) rel;
-      return new EnumerableValuesRel(
-          valuesRel.getCluster(),
-          valuesRel.getRowType(),
-          valuesRel.getTuples(),
-          valuesRel.getTraitSet().replace(EnumerableConvention.INSTANCE));
-    }
-  }
-
   public static final EnumerableOneRowRule ENUMERABLE_ONE_ROW_RULE =
       new EnumerableOneRowRule();
 
-  public static class EnumerableOneRowRule extends ConverterRule {
-    private EnumerableOneRowRule() {
-      super(OneRowRel.class,
-          Convention.NONE,
-          EnumerableConvention.INSTANCE,
-          "EnumerableOneRowRule");
-    }
-
-    public RelNode convert(RelNode rel) {
-      OneRowRel oneRow = (OneRowRel) rel;
-      RexBuilder rexBuilder = rel.getCluster().getRexBuilder();
-      return new EnumerableValuesRel(
-          oneRow.getCluster(),
-          oneRow.getRowType(),
-          Collections.singletonList(
-              Collections.singletonList(
-                  rexBuilder.makeExactLiteral(BigDecimal.ZERO))),
-          oneRow.getTraitSet().replace(EnumerableConvention.INSTANCE));
-    }
-  }
-
   public static final EnumerableEmptyRule ENUMERABLE_EMPTY_RULE =
       new EnumerableEmptyRule();
 
-  public static class EnumerableEmptyRule extends ConverterRule {
-    private EnumerableEmptyRule() {
-      super(EmptyRel.class,
-          Convention.NONE,
-          EnumerableConvention.INSTANCE,
-          "EnumerableEmptyRule");
-    }
-
-    public RelNode convert(RelNode rel) {
-      EmptyRel empty = (EmptyRel) rel;
-      return new EnumerableValuesRel(
-          empty.getCluster(),
-          empty.getRowType(),
-          ImmutableList.<List<RexLiteral>>of(),
-          empty.getTraitSet().replace(EnumerableConvention.INSTANCE));
-    }
-  }
-
-  /** Implementation of {@link org.eigenbase.rel.ValuesRel} in
-   * {@link EnumerableConvention enumerable calling convention}. */
-  public static class EnumerableValuesRel
-      extends ValuesRelBase
-      implements EnumerableRel {
-    EnumerableValuesRel(
-        RelOptCluster cluster,
-        RelDataType rowType,
-        List<List<RexLiteral>> tuples,
-        RelTraitSet traitSet) {
-      super(cluster, rowType, tuples, traitSet);
-    }
-
-    @Override
-    public RelNode copy(
-        RelTraitSet traitSet, List<RelNode> inputs) {
-      assert inputs.isEmpty();
-      return new EnumerableValuesRel(
-          getCluster(), rowType, tuples, traitSet);
-    }
-
-    public Result implement(EnumerableRelImplementor implementor, Prefer pref) {
-/*
-            return Linq4j.asEnumerable(
-                new Object[][] {
-                    new Object[] {1, 2},
-                    new Object[] {3, 4}
-                });
-*/
-      final JavaTypeFactory typeFactory =
-          (JavaTypeFactory) getCluster().getTypeFactory();
-      final BlockBuilder builder = new BlockBuilder();
-      final PhysType physType =
-          PhysTypeImpl.of(
-              implementor.getTypeFactory(),
-              getRowType(),
-              pref.preferCustom());
-      final Type rowClass = physType.getJavaRowType();
-
-      final List<Expression> expressions = new ArrayList<Expression>();
-      final List<RelDataTypeField> fields = rowType.getFieldList();
-      for (List<RexLiteral> tuple : tuples) {
-        final List<Expression> literals = new ArrayList<Expression>();
-        for (Pair<RelDataTypeField, RexLiteral> pair
-            : Pair.zip(fields, tuple)) {
-          literals.add(
-              RexToLixTranslator.translateLiteral(
-                  pair.right,
-                  pair.left.getType(),
-                  typeFactory,
-                  RexImpTable.NullAs.NULL));
-        }
-        expressions.add(physType.record(literals));
-      }
-      builder.add(
-          Expressions.return_(
-              null,
-              Expressions.call(
-                  BuiltinMethod.AS_ENUMERABLE.method,
-                  Expressions.newArrayInit(
-                      Primitive.box(rowClass), expressions))));
-      return implementor.result(physType, builder.toBlock());
-    }
-  }
-
   public static final EnumerableWindowRule ENUMERABLE_WINDOW_RULE =
       new EnumerableWindowRule();
 
-  /**
-   * Rule to convert an {@link org.eigenbase.rel.AggregateRel} to an
-   * {@link net.hydromatic.optiq.rules.java.JavaRules.EnumerableAggregateRel}.
-   */
-  private static class EnumerableWindowRule
-      extends ConverterRule {
-    private EnumerableWindowRule() {
-      super(WindowRel.class, Convention.NONE, EnumerableConvention.INSTANCE,
-          "EnumerableWindowRule");
-    }
-
-    public RelNode convert(RelNode rel) {
-      final WindowRel winAgg = (WindowRel) rel;
-      final RelTraitSet traitSet =
-          winAgg.getTraitSet().replace(EnumerableConvention.INSTANCE);
-      final RelNode child = winAgg.getChild();
-      final RelNode convertedChild =
-          convert(child,
-              child.getTraitSet().replace(EnumerableConvention.INSTANCE));
-      return new EnumerableWindowRel(rel.getCluster(), traitSet, convertedChild,
-          winAgg.getConstants(), winAgg.getRowType(), winAgg.windows);
-    }
-  }
-
-  /** Implementation of {@link org.eigenbase.rel.WindowRel} in
-   * {@link EnumerableConvention enumerable calling convention}. */
-  public static class EnumerableWindowRel extends WindowRelBase
-      implements EnumerableRel {
-    /** Creates an EnumerableWindowRel. */
-    EnumerableWindowRel(
-        RelOptCluster cluster,
-        RelTraitSet traits,
-        RelNode child,
-        List<RexLiteral> constants,
-        RelDataType rowType,
-        List<Window> windows) {
-      super(cluster, traits, child, constants, rowType, windows);
-    }
-
-    @Override
-    public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
-      return new EnumerableWindowRel(getCluster(), traitSet, sole(inputs),
-          constants, rowType, windows);
-    }
-
-    public RelOptCost computeSelfCost(RelOptPlanner planner) {
-      // Cost is proportional to the number of rows and the number of
-      // components (windows and aggregate functions). There is
-      // no I/O cost.
-      //
-      // TODO #1. Add memory cost.
-      // TODO #2. MIN and MAX have higher CPU cost than SUM and COUNT.
-      final double rowsIn = RelMetadataQuery.getRowCount(getChild());
-      int count = windows.size();
-      for (WindowRel.Window window : windows) {
-        count += window.aggCalls.size();
-      }
-      return planner.getCostFactory().makeCost(rowsIn, rowsIn * count, 0);
-    }
-
-    private static class WindowRelInputGetter implements
-        RexToLixTranslator.InputGetter {
-      private final Expression row;
-      private final PhysType rowPhysType;
-      private final int actualInputFieldCount;
-      private final List<Expression> constants;
-
-      private WindowRelInputGetter(Expression row,
-          PhysType rowPhysType, int actualInputFieldCount,
-          List<Expression> constants) {
-        this.row = row;
-        this.rowPhysType = rowPhysType;
-        this.actualInputFieldCount = actualInputFieldCount;
-        this.constants = constants;
-      }
-
-      public Expression field(BlockBuilder list, int index, Type storageType) {
-        if (index < actualInputFieldCount) {
-          Expression current = list.append("current", row);
-          return rowPhysType.fieldReference(current, index, storageType);
-        }
-        return constants.get(index - actualInputFieldCount);
-      }
-    }
-
-
-    private void sampleOfTheGeneratedWindowedAggregate() {
-      // Here's overview of the generated code
-      // For each list of rows that have the same partitioning key, evaluate
-      // all of the windowed aggregate functions.
-
-      // builder
-      Iterator<Integer[]> iterator = null;
-
-      // builder3
-      Integer[] rows = iterator.next();
-
-      int prevStart = -1;
-      int prevEnd = -1;
-
-      for (int i = 0; i < rows.length; i++) {
-        // builder4
-        Integer row = rows[i];
-
-        int start = 0;
-        int end = 100;
-        if (start != prevStart || end != prevEnd) {
-          // builder5
-          int actualStart = 0;
-          if (start != prevStart || end < prevEnd) {
-            // builder6
-            // recompute
-            actualStart = start;
-            // implementReset
-          } else { // must be start == prevStart && end > prevEnd
-            actualStart = prevEnd + 1;
-          }
-          prevStart = start;
-          prevEnd = end;
-
-          if (start != -1) {
-            for (int j = actualStart; j <= end; j++) {
-              // builder7
-              // implementAdd
-            }
-          }
-          // implementResult
-          // list.add(new Xxx(row.deptno, row.empid, sum, count));
-        }
-      }
-      // multiMap.clear(); // allows gc
-      // source = Linq4j.asEnumerable(list);
-    }
-
-    public Result implement(EnumerableRelImplementor implementor, Prefer pref) {
-      final JavaTypeFactory typeFactory = implementor.getTypeFactory();
-      final EnumerableRel child = (EnumerableRel) getChild();
-      final BlockBuilder builder = new BlockBuilder();
-      final Result result = implementor.visitChild(this, 0, child, pref);
-      Expression source_ = builder.append("source", result.block);
-
-      final List<Expression> translatedConstants =
-          new ArrayList<Expression>(constants.size());
-      for (RexLiteral constant : constants) {
-        translatedConstants.add(RexToLixTranslator.translateLiteral(
-            constant, constant.getType(),
-            typeFactory,
-            RexImpTable.NullAs.NULL));
-      }
-
-      PhysType inputPhysType = result.physType;
-
-      ParameterExpression prevStart =
-          Expressions.parameter(int.class, builder.newName("prevStart"));
-      ParameterExpression prevEnd =
-          Expressions.parameter(int.class, builder.newName("prevEnd"));
-
-      builder.add(Expressions.declare(0, prevStart, null));
-      builder.add(Expressions.declare(0, prevEnd, null));
-
-      for (int windowIdx = 0; windowIdx < windows.size(); windowIdx++) {
-        Window window = windows.get(windowIdx);
-        // Comparator:
-        // final Comparator<JdbcTest.Employee> comparator =
-        //    new Comparator<JdbcTest.Employee>() {
-        //      public int compare(JdbcTest.Employee o1,
-        //          JdbcTest.Employee o2) {
-        //        return Integer.compare(o1.empid, o2.empid);
-        //      }
-        //    };
-        final Expression comparator_ =
-            builder.append(
-                "comparator",
-                inputPhysType.generateComparator(
-                    window.collation()));
-
-        Pair<Expression, Expression> partitionIterator =
-            getPartitionIterator(builder, source_, inputPhysType, window,
-                comparator_);
-        final Expression collectionExpr = partitionIterator.left;
-        final Expression iterator_ = partitionIterator.right;
-
-        List<AggImpState> aggs = new ArrayList<AggImpState>();
-        List<AggregateCall> aggregateCalls = window.getAggregateCalls(this);
-        for (int aggIdx = 0; aggIdx < aggregateCalls.size(); aggIdx++) {
-          AggregateCall call = aggregateCalls.get(aggIdx);
-          aggs.add(new AggImpState(aggIdx, call, true));
-        }
-
-        // The output from this stage is the input plus the aggregate functions.
-        final RelDataTypeFactory.FieldInfoBuilder typeBuilder =
-            typeFactory.builder();
-        typeBuilder.addAll(inputPhysType.getRowType().getFieldList());
-        for (AggImpState agg : aggs) {
-          typeBuilder.add(agg.call.name, agg.call.type);
-        }
-        RelDataType outputRowType = typeBuilder.build();
-        final PhysType outputPhysType =
-            PhysTypeImpl.of(
-                typeFactory, outputRowType, pref.prefer(result.format));
-
-        final Expression list_ =
-            builder.append(
-                "list",
-                Expressions.new_(
-                    ArrayList.class,
-                    Expressions.call(
-                        collectionExpr, BuiltinMethod.COLLECTION_SIZE.method)),
-                false);
-
-        Pair<Expression, Expression> collationKey =
-            getRowCollationKey(builder, inputPhysType, window, windowIdx);
-        Expression keySelector = collationKey.left;
-        Expression keyComparator = collationKey.right;
-        final BlockBuilder builder3 = new BlockBuilder();
-        final Expression rows_ =
-            builder3.append(
-                "rows",
-                Expressions.convert_(
-                    Expressions.call(
-                        iterator_, BuiltinMethod.ITERATOR_NEXT.method),
-                    Object[].class),
-                false);
-
-        builder3.add(Expressions.statement(
-            Expressions.assign(prevStart, Expressions.constant(-1))));
-        builder3.add(Expressions.statement(
-            Expressions.assign(prevEnd,
-                Expressions.constant(Integer.MAX_VALUE))));
-
-        final BlockBuilder builder4 = new BlockBuilder();
-
-        final ParameterExpression i_ =
-            Expressions.parameter(int.class, builder4.newName("i"));
-
-        final Expression row_ =
-            builder4.append(
-                "row",
-                RexToLixTranslator.convert(
-                    Expressions.arrayIndex(rows_, i_),
-                    inputPhysType.getJavaRowType()));
-
-        final RexToLixTranslator.InputGetter inputGetter =
-            new WindowRelInputGetter(row_, inputPhysType,
-                result.physType.getRowType().getFieldCount(),
-                translatedConstants);
-
-        final RexToLixTranslator translator =
-            RexToLixTranslator.forAggregation(typeFactory, builder4,
-                inputGetter);
-
-        final List<Expression> outputRow = new ArrayList<Expression>();
-        int fieldCountWithAggResults =
-          inputPhysType.getRowType().getFieldCount();
-        for (int i = 0; i < fieldCountWithAggResults; i++) {
-          outputRow.add(
-              inputPhysType.fieldReference(
-                  row_, i,
-    

<TRUNCATED>

[09/58] [abbrv] [partial] incubator-calcite git commit: [CALCITE-306] Standardize code style for "import package.*; "

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/tools/PlannerTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/tools/PlannerTest.java b/core/src/test/java/org/apache/calcite/tools/PlannerTest.java
index efd16e3..db202f8 100644
--- a/core/src/test/java/org/apache/calcite/tools/PlannerTest.java
+++ b/core/src/test/java/org/apache/calcite/tools/PlannerTest.java
@@ -14,50 +14,79 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.tools;
-
-import net.hydromatic.optiq.SchemaPlus;
-import net.hydromatic.optiq.config.Lex;
-import net.hydromatic.optiq.impl.java.ReflectiveSchema;
-import net.hydromatic.optiq.impl.jdbc.*;
-import net.hydromatic.optiq.impl.jdbc.JdbcRules.JdbcProjectRel;
-import net.hydromatic.optiq.rules.java.EnumerableConvention;
-import net.hydromatic.optiq.rules.java.JavaRules;
-import net.hydromatic.optiq.rules.java.JavaRules.EnumerableProjectRel;
-import net.hydromatic.optiq.test.OptiqAssert;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.rel.convert.ConverterRule;
-import org.eigenbase.rel.metadata.RelMetadataQuery;
-import org.eigenbase.rel.rules.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.reltype.RelDataTypeFactory;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.fun.SqlStdOperatorTable;
-import org.eigenbase.sql.parser.SqlParseException;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.sql.util.ChainedSqlOperatorTable;
-import org.eigenbase.sql.util.ListSqlOperatorTable;
-import org.eigenbase.sql.validate.SqlValidator;
-import org.eigenbase.sql.validate.SqlValidatorScope;
-import org.eigenbase.util.Util;
+package org.apache.calcite.tools;
+
+import org.apache.calcite.adapter.enumerable.EnumerableConvention;
+import org.apache.calcite.adapter.enumerable.EnumerableProject;
+import org.apache.calcite.adapter.enumerable.EnumerableRules;
+import org.apache.calcite.adapter.enumerable.EnumerableTableScan;
+import org.apache.calcite.adapter.java.ReflectiveSchema;
+import org.apache.calcite.adapter.jdbc.JdbcConvention;
+import org.apache.calcite.adapter.jdbc.JdbcImplementor;
+import org.apache.calcite.adapter.jdbc.JdbcRel;
+import org.apache.calcite.adapter.jdbc.JdbcRules;
+import org.apache.calcite.config.Lex;
+import org.apache.calcite.plan.ConventionTraitDef;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelOptPredicateList;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.RelTraitDef;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rel.rules.FilterMergeRule;
+import org.apache.calcite.rel.rules.SortRemoveRule;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.sql.SqlAggFunction;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlDialect;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlOperatorTable;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.parser.SqlParseException;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.ReturnTypes;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.util.ChainedSqlOperatorTable;
+import org.apache.calcite.sql.util.ListSqlOperatorTable;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql.validate.SqlValidatorScope;
+import org.apache.calcite.test.CalciteAssert;
+import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
 
 import org.junit.Test;
 
-import java.util.*;
+import java.util.ArrayList;
+import java.util.List;
 
-import static org.hamcrest.CoreMatchers.*;
-import static org.junit.Assert.*;
+import static org.hamcrest.CoreMatchers.containsString;
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.CoreMatchers.notNullValue;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
 
 /**
  * Unit tests for {@link Planner}.
  */
 public class PlannerTest {
   private void checkParseAndConvert(String query,
-    String queryFromParseTree, String expectedRelExpr) throws Exception {
+      String queryFromParseTree, String expectedRelExpr) throws Exception {
     Planner planner = getPlanner(null);
     SqlNode parse = planner.parse(query);
     assertThat(Util.toLinux(parse.toString()), equalTo(queryFromParseTree));
@@ -75,9 +104,9 @@ public class PlannerTest {
             + "FROM `emps`\n"
             + "WHERE `name` LIKE '%e%'",
 
-        "ProjectRel(empid=[$0], deptno=[$1], name=[$2], salary=[$3], commission=[$4])\n"
-            + "  FilterRel(condition=[LIKE($2, '%e%')])\n"
-            + "    EnumerableTableAccessRel(table=[[hr, emps]])\n");
+        "LogicalProject(empid=[$0], deptno=[$1], name=[$2], salary=[$3], commission=[$4])\n"
+            + "  LogicalFilter(condition=[LIKE($2, '%e%')])\n"
+            + "    EnumerableTableScan(table=[[hr, emps]])\n");
   }
 
   /** Unit test that parses, validates and converts the query using
@@ -85,16 +114,16 @@ public class PlannerTest {
   @Test public void testParseAndConvertWithOrderByAndOffset() throws Exception {
     checkParseAndConvert(
         "select * from \"emps\" "
-        + "order by \"emps\".\"deptno\" offset 10",
+            + "order by \"emps\".\"deptno\" offset 10",
 
         "SELECT *\n"
-        + "FROM `emps`\n"
-        + "ORDER BY `emps`.`deptno`\n"
-        + "OFFSET 10 ROWS",
+            + "FROM `emps`\n"
+            + "ORDER BY `emps`.`deptno`\n"
+            + "OFFSET 10 ROWS",
 
-        "SortRel(sort0=[$1], dir0=[ASC], offset=[10])\n"
-        + "  ProjectRel(empid=[$0], deptno=[$1], name=[$2], salary=[$3], commission=[$4])\n"
-        + "    EnumerableTableAccessRel(table=[[hr, emps]])\n");
+        "Sort(sort0=[$1], dir0=[ASC], offset=[10])\n"
+            + "  LogicalProject(empid=[$0], deptno=[$1], name=[$2], salary=[$3], commission=[$4])\n"
+            + "    EnumerableTableScan(table=[[hr, emps]])\n");
   }
 
   private String toString(RelNode rel) {
@@ -119,10 +148,10 @@ public class PlannerTest {
     Planner planner = getPlanner(null);
     SqlNode parse =
         planner.parse("select * from \"emps\" where \"Xname\" like '%e%'");
-    assertThat(Util.toLinux(parse.toString()), equalTo(
-        "SELECT *\n"
-        + "FROM `emps`\n"
-        + "WHERE `Xname` LIKE '%e%'"));
+    assertThat(Util.toLinux(parse.toString()),
+        equalTo("SELECT *\n"
+            + "FROM `emps`\n"
+            + "WHERE `Xname` LIKE '%e%'"));
 
     try {
       SqlNode validate = planner.validate(parse);
@@ -143,7 +172,7 @@ public class PlannerTest {
     final SchemaPlus rootSchema = Frameworks.createRootSchema(true);
     final FrameworkConfig config = Frameworks.newConfigBuilder()
         .defaultSchema(
-            OptiqAssert.addSchema(rootSchema, OptiqAssert.SchemaSpec.HR))
+            CalciteAssert.addSchema(rootSchema, CalciteAssert.SchemaSpec.HR))
         .operatorTable(opTab)
         .build();
     final Planner planner = Frameworks.getPlanner(config);
@@ -151,8 +180,7 @@ public class PlannerTest {
         planner.parse("select \"deptno\", my_count(\"empid\") from \"emps\"\n"
             + "group by \"deptno\"");
     assertThat(Util.toLinux(parse.toString()),
-        equalTo(
-            "SELECT `deptno`, `MY_COUNT`(`empid`)\n"
+        equalTo("SELECT `deptno`, `MY_COUNT`(`empid`)\n"
             + "FROM `emps`\n"
             + "GROUP BY `deptno`"));
 
@@ -180,7 +208,7 @@ public class PlannerTest {
     final FrameworkConfig config = Frameworks.newConfigBuilder()
         .lex(Lex.ORACLE)
         .defaultSchema(
-            OptiqAssert.addSchema(rootSchema, OptiqAssert.SchemaSpec.HR))
+            CalciteAssert.addSchema(rootSchema, CalciteAssert.SchemaSpec.HR))
         .traitDefs(traitDefs)
         .programs(programs)
         .build();
@@ -188,8 +216,8 @@ public class PlannerTest {
   }
 
   /** Tests that planner throws an error if you pass to
-   * {@link Planner#convert(org.eigenbase.sql.SqlNode)}
-   * a {@link org.eigenbase.sql.SqlNode} that has been parsed but not
+   * {@link Planner#convert(org.apache.calcite.sql.SqlNode)}
+   * a {@link org.apache.calcite.sql.SqlNode} that has been parsed but not
    * validated. */
   @Test public void testConvertWithoutValidateFails() throws Exception {
     Planner planner = getPlanner(null);
@@ -221,8 +249,8 @@ public class PlannerTest {
   @Test public void testMetadataUnionPredicates() throws Exception {
     checkMetadataUnionPredicates(
         "select * from \"emps\" where \"deptno\" < 10\n"
-        + "union all\n"
-        + "select * from \"emps\" where \"empid\" > 2",
+            + "union all\n"
+            + "select * from \"emps\" where \"empid\" > 2",
         "[OR(<($1, 10), >($0, 2))]");
   }
 
@@ -232,8 +260,8 @@ public class PlannerTest {
   @Test public void testMetadataUnionPredicates2() throws Exception {
     checkMetadataUnionPredicates(
         "select * from \"emps\" where \"deptno\" < 10\n"
-        + "union all\n"
-        + "select * from \"emps\"",
+            + "union all\n"
+            + "select * from \"emps\"",
         "[]");
   }
 
@@ -242,16 +270,16 @@ public class PlannerTest {
     // could be simplified to [<($1, 10)] but is nevertheless correct.
     checkMetadataUnionPredicates(
         "select * from \"emps\" where \"deptno\" < 10\n"
-        + "union all\n"
-        + "select * from \"emps\" where \"deptno\" < 10 and \"empid\" > 1",
+            + "union all\n"
+            + "select * from \"emps\" where \"deptno\" < 10 and \"empid\" > 1",
         "[OR(<($1, 10), AND(<($1, 10), >($0, 1)))]");
   }
 
   @Test public void testMetadataUnionPredicates4() throws Exception {
     checkMetadataUnionPredicates(
         "select * from \"emps\" where \"deptno\" < 10\n"
-        + "union all\n"
-        + "select * from \"emps\" where \"deptno\" < 10 or \"empid\" > 1",
+            + "union all\n"
+            + "select * from \"emps\" where \"deptno\" < 10 or \"empid\" > 1",
         "[OR(<($1, 10), <($1, 10), >($0, 1))]");
   }
 
@@ -259,9 +287,9 @@ public class PlannerTest {
   @Test public void testPlan() throws Exception {
     Program program =
         Programs.ofRules(
-            MergeFilterRule.INSTANCE,
-            JavaRules.ENUMERABLE_FILTER_RULE,
-            JavaRules.ENUMERABLE_PROJECT_RULE);
+            FilterMergeRule.INSTANCE,
+            EnumerableRules.ENUMERABLE_FILTER_RULE,
+            EnumerableRules.ENUMERABLE_PROJECT_RULE);
     Planner planner = getPlanner(null, program);
     SqlNode parse = planner.parse("select * from \"emps\"");
     SqlNode validate = planner.validate(parse);
@@ -270,8 +298,8 @@ public class PlannerTest {
         .replace(EnumerableConvention.INSTANCE);
     RelNode transform = planner.transform(0, traitSet, convert);
     assertThat(toString(transform), equalTo(
-        "EnumerableProjectRel(empid=[$0], deptno=[$1], name=[$2], salary=[$3], commission=[$4])\n"
-        + "  EnumerableTableAccessRel(table=[[hr, emps]])\n"));
+        "EnumerableProject(empid=[$0], deptno=[$1], name=[$2], salary=[$3], commission=[$4])\n"
+            + "  EnumerableTableScan(table=[[hr, emps]])\n"));
   }
 
   /** Unit test that parses, validates, converts and
@@ -279,9 +307,9 @@ public class PlannerTest {
   @Test public void testSortPlan() throws Exception {
     RuleSet ruleSet =
         RuleSets.ofList(
-            RemoveSortRule.INSTANCE,
-            JavaRules.ENUMERABLE_PROJECT_RULE,
-            JavaRules.ENUMERABLE_SORT_RULE);
+            SortRemoveRule.INSTANCE,
+            EnumerableRules.ENUMERABLE_PROJECT_RULE,
+            EnumerableRules.ENUMERABLE_SORT_RULE);
     Planner planner = getPlanner(null, Programs.of(ruleSet));
     SqlNode parse = planner.parse(
         "select * from \"emps\" "
@@ -292,38 +320,38 @@ public class PlannerTest {
         .replace(EnumerableConvention.INSTANCE);
     RelNode transform = planner.transform(0, traitSet, convert);
     assertThat(toString(transform), equalTo(
-        "EnumerableSortRel(sort0=[$1], dir0=[ASC])\n"
-        + "  EnumerableProjectRel(empid=[$0], deptno=[$1], name=[$2], salary=[$3], commission=[$4])\n"
-        + "    EnumerableTableAccessRel(table=[[hr, emps]])\n"));
+        "EnumerableSort(sort0=[$1], dir0=[ASC])\n"
+            + "  EnumerableProject(empid=[$0], deptno=[$1], name=[$2], salary=[$3], commission=[$4])\n"
+            + "    EnumerableTableScan(table=[[hr, emps]])\n"));
   }
 
   /** Unit test that parses, validates, converts and
    * plans for query using two duplicate order by.
-   * The duplicate order by should be removed by RemoveSortRule*/
+   * The duplicate order by should be removed by SortRemoveRule*/
   @Test public void testDuplicateSortPlan() throws Exception {
     RuleSet ruleSet =
         RuleSets.ofList(
-            RemoveSortRule.INSTANCE,
-            JavaRules.ENUMERABLE_PROJECT_RULE,
-            JavaRules.ENUMERABLE_SORT_RULE);
+            SortRemoveRule.INSTANCE,
+            EnumerableRules.ENUMERABLE_PROJECT_RULE,
+            EnumerableRules.ENUMERABLE_SORT_RULE);
     Planner planner = getPlanner(null, Programs.of(ruleSet));
     SqlNode parse = planner.parse(
         "select \"empid\" from ( "
-         + "select * "
-         + "from \"emps\" "
-         + "order by \"emps\".\"deptno\") "
-         + "order by \"deptno\"");
+            + "select * "
+            + "from \"emps\" "
+            + "order by \"emps\".\"deptno\") "
+            + "order by \"deptno\"");
     SqlNode validate = planner.validate(parse);
     RelNode convert = planner.convert(validate);
     RelTraitSet traitSet = planner.getEmptyTraitSet()
         .replace(EnumerableConvention.INSTANCE);
     RelNode transform = planner.transform(0, traitSet, convert);
     assertThat(toString(transform), equalTo(
-        "EnumerableProjectRel(empid=[$0])\n"
-        + "  EnumerableProjectRel(empid=[$0], deptno=[$1])\n"
-        + "    EnumerableSortRel(sort0=[$1], dir0=[ASC])\n"
-        + "      EnumerableProjectRel(empid=[$0], deptno=[$1], name=[$2], salary=[$3], commission=[$4])\n"
-        + "        EnumerableTableAccessRel(table=[[hr, emps]])\n"));
+        "EnumerableProject(empid=[$0])\n"
+            + "  EnumerableProject(empid=[$0], deptno=[$1])\n"
+            + "    EnumerableSort(sort0=[$1], dir0=[ASC])\n"
+            + "      EnumerableProject(empid=[$0], deptno=[$1], name=[$2], salary=[$3], commission=[$4])\n"
+            + "        EnumerableTableScan(table=[[hr, emps]])\n"));
   }
 
   /** Unit test that parses, validates, converts and
@@ -331,8 +359,8 @@ public class PlannerTest {
   @Test public void testDuplicateSortPlanWORemoveSortRule() throws Exception {
     RuleSet ruleSet =
         RuleSets.ofList(
-            JavaRules.ENUMERABLE_PROJECT_RULE,
-            JavaRules.ENUMERABLE_SORT_RULE);
+            EnumerableRules.ENUMERABLE_PROJECT_RULE,
+            EnumerableRules.ENUMERABLE_SORT_RULE);
     Planner planner = getPlanner(null, Programs.of(ruleSet));
     SqlNode parse = planner.parse(
         "select \"empid\" from ( "
@@ -346,12 +374,12 @@ public class PlannerTest {
         .replace(EnumerableConvention.INSTANCE);
     RelNode transform = planner.transform(0, traitSet, convert);
     assertThat(toString(transform), equalTo(
-        "EnumerableProjectRel(empid=[$0])\n"
-        + "  EnumerableSortRel(sort0=[$1], dir0=[ASC])\n"
-        + "    EnumerableProjectRel(empid=[$0], deptno=[$1])\n"
-        + "      EnumerableSortRel(sort0=[$1], dir0=[ASC])\n"
-        + "        EnumerableProjectRel(empid=[$0], deptno=[$1], name=[$2], salary=[$3], commission=[$4])\n"
-        + "          EnumerableTableAccessRel(table=[[hr, emps]])\n"));
+        "EnumerableProject(empid=[$0])\n"
+            + "  EnumerableSort(sort0=[$1], dir0=[ASC])\n"
+            + "    EnumerableProject(empid=[$0], deptno=[$1])\n"
+            + "      EnumerableSort(sort0=[$1], dir0=[ASC])\n"
+            + "        EnumerableProject(empid=[$0], deptno=[$1], name=[$2], salary=[$3], commission=[$4])\n"
+            + "          EnumerableTableScan(table=[[hr, emps]])\n"));
   }
 
   /** Unit test that parses, validates, converts and plans. Planner is
@@ -359,9 +387,9 @@ public class PlannerTest {
   @Test public void testPlanWithExplicitTraitDefs() throws Exception {
     RuleSet ruleSet =
         RuleSets.ofList(
-            MergeFilterRule.INSTANCE,
-            JavaRules.ENUMERABLE_FILTER_RULE,
-            JavaRules.ENUMERABLE_PROJECT_RULE);
+            FilterMergeRule.INSTANCE,
+            EnumerableRules.ENUMERABLE_FILTER_RULE,
+            EnumerableRules.ENUMERABLE_PROJECT_RULE);
     final List<RelTraitDef> traitDefs = new ArrayList<RelTraitDef>();
     traitDefs.add(ConventionTraitDef.INSTANCE);
     traitDefs.add(RelCollationTraitDef.INSTANCE);
@@ -375,17 +403,17 @@ public class PlannerTest {
         .replace(EnumerableConvention.INSTANCE);
     RelNode transform = planner.transform(0, traitSet, convert);
     assertThat(toString(transform), equalTo(
-        "EnumerableProjectRel(empid=[$0], deptno=[$1], name=[$2], salary=[$3], commission=[$4])\n"
-        + "  EnumerableTableAccessRel(table=[[hr, emps]])\n"));
+        "EnumerableProject(empid=[$0], deptno=[$1], name=[$2], salary=[$3], commission=[$4])\n"
+            + "  EnumerableTableScan(table=[[hr, emps]])\n"));
   }
 
   /** Unit test that calls {@link Planner#transform} twice. */
   @Test public void testPlanTransformTwice() throws Exception {
     RuleSet ruleSet =
         RuleSets.ofList(
-            MergeFilterRule.INSTANCE,
-            JavaRules.ENUMERABLE_FILTER_RULE,
-            JavaRules.ENUMERABLE_PROJECT_RULE);
+            FilterMergeRule.INSTANCE,
+            EnumerableRules.ENUMERABLE_FILTER_RULE,
+            EnumerableRules.ENUMERABLE_PROJECT_RULE);
     Planner planner = getPlanner(null, Programs.of(ruleSet));
     SqlNode parse = planner.parse("select * from \"emps\"");
     SqlNode validate = planner.validate(parse);
@@ -395,8 +423,8 @@ public class PlannerTest {
     RelNode transform = planner.transform(0, traitSet, convert);
     RelNode transform2 = planner.transform(0, traitSet, transform);
     assertThat(toString(transform2), equalTo(
-        "EnumerableProjectRel(empid=[$0], deptno=[$1], name=[$2], salary=[$3], commission=[$4])\n"
-        + "  EnumerableTableAccessRel(table=[[hr, emps]])\n"));
+        "EnumerableProject(empid=[$0], deptno=[$1], name=[$2], salary=[$3], commission=[$4])\n"
+            + "  EnumerableTableScan(table=[[hr, emps]])\n"));
   }
 
   /** Tests that Hive dialect does not generate "AS". */
@@ -417,7 +445,7 @@ public class PlannerTest {
   /** Unit test that calls {@link Planner#transform} twice,
    * with different rule sets, with different conventions.
    *
-   * <p>{@link net.hydromatic.optiq.impl.jdbc.JdbcConvention} is different
+   * <p>{@link org.apache.calcite.adapter.jdbc.JdbcConvention} is different
    * from the typical convention in that it is not a singleton. Switching to
    * a different instance causes problems unless planner state is wiped clean
    * between calls to {@link Planner#transform}. */
@@ -425,9 +453,9 @@ public class PlannerTest {
       throws Exception {
     Program program0 =
         Programs.ofRules(
-            MergeFilterRule.INSTANCE,
-            JavaRules.ENUMERABLE_FILTER_RULE,
-            JavaRules.ENUMERABLE_PROJECT_RULE);
+            FilterMergeRule.INSTANCE,
+            EnumerableRules.ENUMERABLE_FILTER_RULE,
+            EnumerableRules.ENUMERABLE_PROJECT_RULE);
 
     JdbcConvention out = new JdbcConvention(null, null, "myjdbc");
     Program program1 = Programs.ofRules(
@@ -448,8 +476,8 @@ public class PlannerTest {
     RelNode transform = planner.transform(0, traitSet0, convert);
     RelNode transform2 = planner.transform(1, traitSet1, transform);
     assertThat(toString(transform2), equalTo(
-        "JdbcProjectRel(name=[$2])\n"
-        + "  MockJdbcTableScan(table=[[hr, emps]])\n"));
+        "JdbcProject(name=[$2])\n"
+            + "  MockJdbcTableScan(table=[[hr, emps]])\n"));
   }
 
   /** Unit test that plans a query with a large number of joins. */
@@ -476,7 +504,7 @@ public class PlannerTest {
     checkJoinNWay(5); // LoptOptimizeJoinRule disabled; takes about .4s
     checkJoinNWay(9); // LoptOptimizeJoinRule enabled; takes about 0.04s
     checkJoinNWay(35); // takes about 2s
-    if (OptiqAssert.ENABLE_SLOW) {
+    if (CalciteAssert.ENABLE_SLOW) {
       checkJoinNWay(60); // takes about 15s
     }
   }
@@ -503,15 +531,16 @@ public class PlannerTest {
         .replace(EnumerableConvention.INSTANCE);
     RelNode transform = planner.transform(0, traitSet, convert);
     assertThat(toString(transform), containsString(
-        "EnumerableJoinRel(condition=[=($0, $3)], joinType=[inner])"));
+        "EnumerableJoin(condition=[=($0, $3)], joinType=[inner])"));
   }
 
   /** Test case for
    * <a href="https://issues.apache.org/jira/browse/CALCITE-435">CALCITE-435</a>,
    * "LoptOptimizeJoinRule incorrectly re-orders outer joins".
    *
-   * <p>Checks the {@link org.eigenbase.rel.rules.LoptOptimizeJoinRule} on a
-   * query with a left outer join.
+   * <p>Checks the
+   * {@link org.apache.calcite.rel.rules.LoptOptimizeJoinRule} on a query with a
+   * left outer join.
    *
    * <p>Specifically, tests that a relation (dependents) in an inner join
    * cannot be pushed into an outer join (emps left join depts).
@@ -519,16 +548,16 @@ public class PlannerTest {
   @Test public void testHeuristicLeftJoin() throws Exception {
     checkHeuristic(
         "select * from \"emps\" as e\n"
-        + "left join \"depts\" as d using (\"deptno\")\n"
-        + "join \"dependents\" as p on e.\"empid\" = p.\"empid\"",
-        "EnumerableProjectRel(empid=[$0], deptno=[$1], name=[$2], salary=[$3], commission=[$4], deptno0=[$5], name0=[$6], employees=[$7], empid0=[$8], name1=[$9])\n"
-        + "  EnumerableProjectRel(empid=[$2], deptno=[$3], name=[$4], salary=[$5], commission=[$6], deptno0=[$7], name0=[$8], employees=[$9], empid0=[$0], name1=[$1])\n"
-        + "    EnumerableJoinRel(condition=[=($0, $2)], joinType=[inner])\n"
-        + "      EnumerableTableAccessRel(table=[[hr, dependents]])\n"
-        + "      EnumerableProjectRel(empid=[$0], deptno=[$1], name=[$2], salary=[$3], commission=[$4], deptno0=[$5], name0=[$6], employees=[$7])\n"
-        + "        EnumerableJoinRel(condition=[=($1, $5)], joinType=[left])\n"
-        + "          EnumerableTableAccessRel(table=[[hr, emps]])\n"
-        + "          EnumerableTableAccessRel(table=[[hr, depts]])");
+            + "left join \"depts\" as d using (\"deptno\")\n"
+            + "join \"dependents\" as p on e.\"empid\" = p.\"empid\"",
+        "EnumerableProject(empid=[$0], deptno=[$1], name=[$2], salary=[$3], commission=[$4], deptno0=[$5], name0=[$6], employees=[$7], empid0=[$8], name1=[$9])\n"
+            + "  EnumerableProject(empid=[$2], deptno=[$3], name=[$4], salary=[$5], commission=[$6], deptno0=[$7], name0=[$8], employees=[$9], empid0=[$0], name1=[$1])\n"
+            + "    EnumerableJoin(condition=[=($0, $2)], joinType=[inner])\n"
+            + "      EnumerableTableScan(table=[[hr, dependents]])\n"
+            + "      EnumerableProject(empid=[$0], deptno=[$1], name=[$2], salary=[$3], commission=[$4], deptno0=[$5], name0=[$6], employees=[$7])\n"
+            + "        EnumerableJoin(condition=[=($1, $5)], joinType=[left])\n"
+            + "          EnumerableTableScan(table=[[hr, emps]])\n"
+            + "          EnumerableTableScan(table=[[hr, depts]])");
   }
 
   /** It would probably be OK to transform
@@ -540,16 +569,16 @@ public class PlannerTest {
   @Test public void testHeuristicPushInnerJoin() throws Exception {
     checkHeuristic(
         "select * from \"emps\" as e\n"
-        + "right join \"depts\" as d using (\"deptno\")\n"
-        + "join \"dependents\" as p on e.\"empid\" = p.\"empid\"",
-        "EnumerableProjectRel(empid=[$0], deptno=[$1], name=[$2], salary=[$3], commission=[$4], deptno0=[$5], name0=[$6], employees=[$7], empid0=[$8], name1=[$9])\n"
-        + "  EnumerableProjectRel(empid=[$2], deptno=[$3], name=[$4], salary=[$5], commission=[$6], deptno0=[$7], name0=[$8], employees=[$9], empid0=[$0], name1=[$1])\n"
-        + "    EnumerableJoinRel(condition=[=($0, $2)], joinType=[inner])\n"
-        + "      EnumerableTableAccessRel(table=[[hr, dependents]])\n"
-        + "      EnumerableProjectRel(empid=[$0], deptno=[$1], name=[$2], salary=[$3], commission=[$4], deptno0=[$5], name0=[$6], employees=[$7])\n"
-        + "        EnumerableJoinRel(condition=[=($1, $5)], joinType=[right])\n"
-        + "          EnumerableTableAccessRel(table=[[hr, emps]])\n"
-        + "          EnumerableTableAccessRel(table=[[hr, depts]])");
+            + "right join \"depts\" as d using (\"deptno\")\n"
+            + "join \"dependents\" as p on e.\"empid\" = p.\"empid\"",
+        "EnumerableProject(empid=[$0], deptno=[$1], name=[$2], salary=[$3], commission=[$4], deptno0=[$5], name0=[$6], employees=[$7], empid0=[$8], name1=[$9])\n"
+            + "  EnumerableProject(empid=[$2], deptno=[$3], name=[$4], salary=[$5], commission=[$6], deptno0=[$7], name0=[$8], employees=[$9], empid0=[$0], name1=[$1])\n"
+            + "    EnumerableJoin(condition=[=($0, $2)], joinType=[inner])\n"
+            + "      EnumerableTableScan(table=[[hr, dependents]])\n"
+            + "      EnumerableProject(empid=[$0], deptno=[$1], name=[$2], salary=[$3], commission=[$4], deptno0=[$5], name0=[$6], employees=[$7])\n"
+            + "        EnumerableJoin(condition=[=($1, $5)], joinType=[right])\n"
+            + "          EnumerableTableScan(table=[[hr, emps]])\n"
+            + "          EnumerableTableScan(table=[[hr, depts]])");
   }
 
   /** Tests that a relation (dependents) that is on the null-generating side of
@@ -557,16 +586,16 @@ public class PlannerTest {
   @Test public void testHeuristicRightJoin() throws Exception {
     checkHeuristic(
         "select * from \"emps\" as e\n"
-        + "join \"depts\" as d using (\"deptno\")\n"
-        + "right join \"dependents\" as p on e.\"empid\" = p.\"empid\"",
-        "EnumerableProjectRel(empid=[$0], deptno=[$1], name=[$2], salary=[$3], commission=[$4], deptno0=[$5], name0=[$6], employees=[$7], empid0=[$8], name1=[$9])\n"
-        + "  EnumerableProjectRel(empid=[$2], deptno=[$3], name=[$4], salary=[$5], commission=[$6], deptno0=[$7], name0=[$8], employees=[$9], empid0=[$0], name1=[$1])\n"
-        + "    EnumerableJoinRel(condition=[=($0, $2)], joinType=[left])\n"
-        + "      EnumerableTableAccessRel(table=[[hr, dependents]])\n"
-        + "      EnumerableProjectRel(empid=[$0], deptno=[$1], name=[$2], salary=[$3], commission=[$4], deptno0=[$5], name0=[$6], employees=[$7])\n"
-        + "        EnumerableJoinRel(condition=[=($1, $5)], joinType=[inner])\n"
-        + "          EnumerableTableAccessRel(table=[[hr, emps]])\n"
-        + "          EnumerableTableAccessRel(table=[[hr, depts]])");
+            + "join \"depts\" as d using (\"deptno\")\n"
+            + "right join \"dependents\" as p on e.\"empid\" = p.\"empid\"",
+        "EnumerableProject(empid=[$0], deptno=[$1], name=[$2], salary=[$3], commission=[$4], deptno0=[$5], name0=[$6], employees=[$7], empid0=[$8], name1=[$9])\n"
+            + "  EnumerableProject(empid=[$2], deptno=[$3], name=[$4], salary=[$5], commission=[$6], deptno0=[$7], name0=[$8], employees=[$9], empid0=[$0], name1=[$1])\n"
+            + "    EnumerableJoin(condition=[=($0, $2)], joinType=[left])\n"
+            + "      EnumerableTableScan(table=[[hr, dependents]])\n"
+            + "      EnumerableProject(empid=[$0], deptno=[$1], name=[$2], salary=[$3], commission=[$4], deptno0=[$5], name0=[$6], employees=[$7])\n"
+            + "        EnumerableJoin(condition=[=($1, $5)], joinType=[inner])\n"
+            + "          EnumerableTableScan(table=[[hr, emps]])\n"
+            + "          EnumerableTableScan(table=[[hr, depts]])");
   }
 
   private void checkHeuristic(String sql, String expected) throws Exception {
@@ -585,20 +614,20 @@ public class PlannerTest {
    * bushy, but nevertheless exercises the bushy-join heuristic optimizer. */
   @Test public void testAlmostBushy() throws Exception {
     checkBushy("select *\n"
-        + "from \"sales_fact_1997\" as s\n"
-        + "  join \"customer\" as c using (\"customer_id\")\n"
-        + "  join \"product\" as p using (\"product_id\")\n"
-        + "where c.\"city\" = 'San Francisco'\n"
-        + "and p.\"brand_name\" = 'Washington'",
-        "EnumerableProjectRel(product_id=[$0], time_id=[$1], customer_id=[$2], promotion_id=[$3], store_id=[$4], store_sales=[$5], store_cost=[$6], unit_sales=[$7], customer_id0=[$8], account_num=[$9], lname=[$10], fname=[$11], mi=[$12], address1=[$13], address2=[$14], address3=[$15], address4=[$16], city=[$17], state_province=[$18], postal_code=[$19], country=[$20], customer_region_id=[$21], phone1=[$22], phone2=[$23], birthdate=[$24], marital_status=[$25], yearly_income=[$26], gender=[$27], total_children=[$28], num_children_at_home=[$29], education=[$30], date_accnt_opened=[$31], member_card=[$32], occupation=[$33], houseowner=[$34], num_cars_owned=[$35], fullname=[$36], product_class_id=[$37], product_id0=[$38], brand_name=[$39], product_name=[$40], SKU=[$41], SRP=[$42], gross_weight=[$43], net_weight=[$44], recyclable_package=[$45], low_fat=[$46], units_per_case=[$47], cases_per_pallet=[$48], shelf_width=[$49], shelf_height=[$50], shelf_depth=[$51])\n"
-        + "  EnumerableProjectRel($f0=[$44], $f1=[$45], $f2=[$46], $f3=[$47], $f4=[$48], $f5=[$49], $f6=[$50], $f7=[$51], $f8=[$15], $f9=[$16], $f10=[$17], $f11=[$18], $f12=[$19], $f13=[$20], $f14=[$21], $f15=[$22], $f16=[$23], $f17=[$24], $f18=[$25], $f19=[$26], $f20=[$27], $f21=[$28], $f22=[$29], $f23=[$30], $f24=[$31], $f25=[$32], $f26=[$33], $f27=[$34], $f28=[$35], $f29=[$36], $f30=[$37], $f31=[$38], $f32=[$39], $f33=[$40], $f34=[$41], $f35=[$42], $f36=[$43], $f37=[$0], $f38=[$1], $f39=[$2], $f40=[$3], $f41=[$4], $f42=[$5], $f43=[$6], $f44=[$7], $f45=[$8], $f46=[$9], $f47=[$10], $f48=[$11], $f49=[$12], $f50=[$13], $f51=[$14])\n"
-        + "    EnumerableJoinRel(condition=[=($1, $44)], joinType=[inner])\n"
-        + "      EnumerableFilterRel(condition=[=($2, 'Washington')])\n"
-        + "        EnumerableTableAccessRel(table=[[foodmart2, product]])\n"
-        + "      EnumerableJoinRel(condition=[=($0, $31)], joinType=[inner])\n"
-        + "        EnumerableFilterRel(condition=[=($9, 'San Francisco')])\n"
-        + "          EnumerableTableAccessRel(table=[[foodmart2, customer]])\n"
-        + "        EnumerableTableAccessRel(table=[[foodmart2, sales_fact_1997]])\n");
+            + "from \"sales_fact_1997\" as s\n"
+            + "  join \"customer\" as c using (\"customer_id\")\n"
+            + "  join \"product\" as p using (\"product_id\")\n"
+            + "where c.\"city\" = 'San Francisco'\n"
+            + "and p.\"brand_name\" = 'Washington'",
+        "EnumerableProject(product_id=[$0], time_id=[$1], customer_id=[$2], promotion_id=[$3], store_id=[$4], store_sales=[$5], store_cost=[$6], unit_sales=[$7], customer_id0=[$8], account_num=[$9], lname=[$10], fname=[$11], mi=[$12], address1=[$13], address2=[$14], address3=[$15], address4=[$16], city=[$17], state_province=[$18], postal_code=[$19], country=[$20], customer_region_id=[$21], phone1=[$22], phone2=[$23], birthdate=[$24], marital_status=[$25], yearly_income=[$26], gender=[$27], total_children=[$28], num_children_at_home=[$29], education=[$30], date_accnt_opened=[$31], member_card=[$32], occupation=[$33], houseowner=[$34], num_cars_owned=[$35], fullname=[$36], product_class_id=[$37], product_id0=[$38], brand_name=[$39], product_name=[$40], SKU=[$41], SRP=[$42], gross_weight=[$43], net_weight=[$44], recyclable_package=[$45], low_fat=[$46], units_per_case=[$47], cases_per_pallet=[$48], shelf_width=[$49], shelf_height=[$50], shelf_depth=[$51])\n"
+            + "  EnumerableProject($f0=[$44], $f1=[$45], $f2=[$46], $f3=[$47], $f4=[$48], $f5=[$49], $f6=[$50], $f7=[$51], $f8=[$15], $f9=[$16], $f10=[$17], $f11=[$18], $f12=[$19], $f13=[$20], $f14=[$21], $f15=[$22], $f16=[$23], $f17=[$24], $f18=[$25], $f19=[$26], $f20=[$27], $f21=[$28], $f22=[$29], $f23=[$30], $f24=[$31], $f25=[$32], $f26=[$33], $f27=[$34], $f28=[$35], $f29=[$36], $f30=[$37], $f31=[$38], $f32=[$39], $f33=[$40], $f34=[$41], $f35=[$42], $f36=[$43], $f37=[$0], $f38=[$1], $f39=[$2], $f40=[$3], $f41=[$4], $f42=[$5], $f43=[$6], $f44=[$7], $f45=[$8], $f46=[$9], $f47=[$10], $f48=[$11], $f49=[$12], $f50=[$13], $f51=[$14])\n"
+            + "    EnumerableJoin(condition=[=($1, $44)], joinType=[inner])\n"
+            + "      EnumerableFilter(condition=[=($2, 'Washington')])\n"
+            + "        EnumerableTableScan(table=[[foodmart2, product]])\n"
+            + "      EnumerableJoin(condition=[=($0, $31)], joinType=[inner])\n"
+            + "        EnumerableFilter(condition=[=($9, 'San Francisco')])\n"
+            + "          EnumerableTableScan(table=[[foodmart2, customer]])\n"
+            + "        EnumerableTableScan(table=[[foodmart2, sales_fact_1997]])\n");
   }
 
   /** Plans a 4-table join query on the FoodMart schema.
@@ -610,91 +639,91 @@ public class PlannerTest {
    * if you don't assume 'x' is left-associative. */
   @Test public void testBushy() throws Exception {
     checkBushy("select *\n"
-        + "from \"sales_fact_1997\" as s\n"
-        + "  join \"customer\" as c using (\"customer_id\")\n"
-        + "  join \"product\" as p using (\"product_id\")\n"
-        + "  join \"product_class\" as pc using (\"product_class_id\")\n"
-        + "where c.\"city\" = 'San Francisco'\n"
-        + "and p.\"brand_name\" = 'Washington'",
-        "EnumerableProjectRel(product_id=[$0], time_id=[$1], customer_id=[$2], promotion_id=[$3], store_id=[$4], store_sales=[$5], store_cost=[$6], unit_sales=[$7], customer_id0=[$8], account_num=[$9], lname=[$10], fname=[$11], mi=[$12], address1=[$13], address2=[$14], address3=[$15], address4=[$16], city=[$17], state_province=[$18], postal_code=[$19], country=[$20], customer_region_id=[$21], phone1=[$22], phone2=[$23], birthdate=[$24], marital_status=[$25], yearly_income=[$26], gender=[$27], total_children=[$28], num_children_at_home=[$29], education=[$30], date_accnt_opened=[$31], member_card=[$32], occupation=[$33], houseowner=[$34], num_cars_owned=[$35], fullname=[$36], product_class_id=[$37], product_id0=[$38], brand_name=[$39], product_name=[$40], SKU=[$41], SRP=[$42], gross_weight=[$43], net_weight=[$44], recyclable_package=[$45], low_fat=[$46], units_per_case=[$47], cases_per_pallet=[$48], shelf_width=[$49], shelf_height=[$50], shelf_depth=[$51], product_class_id0=[$52], pro
 duct_subcategory=[$53], product_category=[$54], product_department=[$55], product_family=[$56])\n"
-        + "  EnumerableProjectRel($f0=[$49], $f1=[$50], $f2=[$51], $f3=[$52], $f4=[$53], $f5=[$54], $f6=[$55], $f7=[$56], $f8=[$0], $f9=[$1], $f10=[$2], $f11=[$3], $f12=[$4], $f13=[$5], $f14=[$6], $f15=[$7], $f16=[$8], $f17=[$9], $f18=[$10], $f19=[$11], $f20=[$12], $f21=[$13], $f22=[$14], $f23=[$15], $f24=[$16], $f25=[$17], $f26=[$18], $f27=[$19], $f28=[$20], $f29=[$21], $f30=[$22], $f31=[$23], $f32=[$24], $f33=[$25], $f34=[$26], $f35=[$27], $f36=[$28], $f37=[$34], $f38=[$35], $f39=[$36], $f40=[$37], $f41=[$38], $f42=[$39], $f43=[$40], $f44=[$41], $f45=[$42], $f46=[$43], $f47=[$44], $f48=[$45], $f49=[$46], $f50=[$47], $f51=[$48], $f52=[$29], $f53=[$30], $f54=[$31], $f55=[$32], $f56=[$33])\n"
-        + "    EnumerableJoinRel(condition=[=($0, $51)], joinType=[inner])\n"
-        + "      EnumerableFilterRel(condition=[=($9, 'San Francisco')])\n"
-        + "        EnumerableTableAccessRel(table=[[foodmart2, customer]])\n"
-        + "      EnumerableJoinRel(condition=[=($6, $20)], joinType=[inner])\n"
-        + "        EnumerableJoinRel(condition=[=($0, $5)], joinType=[inner])\n"
-        + "          EnumerableTableAccessRel(table=[[foodmart2, product_class]])\n"
-        + "          EnumerableFilterRel(condition=[=($2, 'Washington')])\n"
-        + "            EnumerableTableAccessRel(table=[[foodmart2, product]])\n"
-        + "        EnumerableTableAccessRel(table=[[foodmart2, sales_fact_1997]])\n");
+            + "from \"sales_fact_1997\" as s\n"
+            + "  join \"customer\" as c using (\"customer_id\")\n"
+            + "  join \"product\" as p using (\"product_id\")\n"
+            + "  join \"product_class\" as pc using (\"product_class_id\")\n"
+            + "where c.\"city\" = 'San Francisco'\n"
+            + "and p.\"brand_name\" = 'Washington'",
+        "EnumerableProject(product_id=[$0], time_id=[$1], customer_id=[$2], promotion_id=[$3], store_id=[$4], store_sales=[$5], store_cost=[$6], unit_sales=[$7], customer_id0=[$8], account_num=[$9], lname=[$10], fname=[$11], mi=[$12], address1=[$13], address2=[$14], address3=[$15], address4=[$16], city=[$17], state_province=[$18], postal_code=[$19], country=[$20], customer_region_id=[$21], phone1=[$22], phone2=[$23], birthdate=[$24], marital_status=[$25], yearly_income=[$26], gender=[$27], total_children=[$28], num_children_at_home=[$29], education=[$30], date_accnt_opened=[$31], member_card=[$32], occupation=[$33], houseowner=[$34], num_cars_owned=[$35], fullname=[$36], product_class_id=[$37], product_id0=[$38], brand_name=[$39], product_name=[$40], SKU=[$41], SRP=[$42], gross_weight=[$43], net_weight=[$44], recyclable_package=[$45], low_fat=[$46], units_per_case=[$47], cases_per_pallet=[$48], shelf_width=[$49], shelf_height=[$50], shelf_depth=[$51], product_class_id0=[$52], produc
 t_subcategory=[$53], product_category=[$54], product_department=[$55], product_family=[$56])\n"
+            + "  EnumerableProject($f0=[$49], $f1=[$50], $f2=[$51], $f3=[$52], $f4=[$53], $f5=[$54], $f6=[$55], $f7=[$56], $f8=[$0], $f9=[$1], $f10=[$2], $f11=[$3], $f12=[$4], $f13=[$5], $f14=[$6], $f15=[$7], $f16=[$8], $f17=[$9], $f18=[$10], $f19=[$11], $f20=[$12], $f21=[$13], $f22=[$14], $f23=[$15], $f24=[$16], $f25=[$17], $f26=[$18], $f27=[$19], $f28=[$20], $f29=[$21], $f30=[$22], $f31=[$23], $f32=[$24], $f33=[$25], $f34=[$26], $f35=[$27], $f36=[$28], $f37=[$34], $f38=[$35], $f39=[$36], $f40=[$37], $f41=[$38], $f42=[$39], $f43=[$40], $f44=[$41], $f45=[$42], $f46=[$43], $f47=[$44], $f48=[$45], $f49=[$46], $f50=[$47], $f51=[$48], $f52=[$29], $f53=[$30], $f54=[$31], $f55=[$32], $f56=[$33])\n"
+            + "    EnumerableJoin(condition=[=($0, $51)], joinType=[inner])\n"
+            + "      EnumerableFilter(condition=[=($9, 'San Francisco')])\n"
+            + "        EnumerableTableScan(table=[[foodmart2, customer]])\n"
+            + "      EnumerableJoin(condition=[=($6, $20)], joinType=[inner])\n"
+            + "        EnumerableJoin(condition=[=($0, $5)], joinType=[inner])\n"
+            + "          EnumerableTableScan(table=[[foodmart2, product_class]])\n"
+            + "          EnumerableFilter(condition=[=($2, 'Washington')])\n"
+            + "            EnumerableTableScan(table=[[foodmart2, product]])\n"
+            + "        EnumerableTableScan(table=[[foodmart2, sales_fact_1997]])\n");
   }
 
   /** Plans a 5-table join query on the FoodMart schema. The ideal plan is
    * bushy: store x (customer x (product_class x product x sales)). */
   @Test public void testBushy5() throws Exception {
     checkBushy("select *\n"
-        + "from \"sales_fact_1997\" as s\n"
-        + "  join \"customer\" as c using (\"customer_id\")\n"
-        + "  join \"product\" as p using (\"product_id\")\n"
-        + "  join \"product_class\" as pc using (\"product_class_id\")\n"
-        + "  join \"store\" as st using (\"store_id\")\n"
-        + "where c.\"city\" = 'San Francisco'\n",
-        "EnumerableProjectRel(product_id=[$0], time_id=[$1], customer_id=[$2], promotion_id=[$3], store_id=[$4], store_sales=[$5], store_cost=[$6], unit_sales=[$7], customer_id0=[$8], account_num=[$9], lname=[$10], fname=[$11], mi=[$12], address1=[$13], address2=[$14], address3=[$15], address4=[$16], city=[$17], state_province=[$18], postal_code=[$19], country=[$20], customer_region_id=[$21], phone1=[$22], phone2=[$23], birthdate=[$24], marital_status=[$25], yearly_income=[$26], gender=[$27], total_children=[$28], num_children_at_home=[$29], education=[$30], date_accnt_opened=[$31], member_card=[$32], occupation=[$33], houseowner=[$34], num_cars_owned=[$35], fullname=[$36], product_class_id=[$37], product_id0=[$38], brand_name=[$39], product_name=[$40], SKU=[$41], SRP=[$42], gross_weight=[$43], net_weight=[$44], recyclable_package=[$45], low_fat=[$46], units_per_case=[$47], cases_per_pallet=[$48], shelf_width=[$49], shelf_height=[$50], shelf_depth=[$51], product_class_id0=[$52], pro
 duct_subcategory=[$53], product_category=[$54], product_department=[$55], product_family=[$56], store_id0=[$57], store_type=[$58], region_id=[$59], store_name=[$60], store_number=[$61], store_street_address=[$62], store_city=[$63], store_state=[$64], store_postal_code=[$65], store_country=[$66], store_manager=[$67], store_phone=[$68], store_fax=[$69], first_opened_date=[$70], last_remodel_date=[$71], store_sqft=[$72], grocery_sqft=[$73], frozen_sqft=[$74], meat_sqft=[$75], coffee_bar=[$76], video_store=[$77], salad_bar=[$78], prepared_food=[$79], florist=[$80])\n"
-        + "  EnumerableProjectRel($f0=[$73], $f1=[$74], $f2=[$75], $f3=[$76], $f4=[$77], $f5=[$78], $f6=[$79], $f7=[$80], $f8=[$24], $f9=[$25], $f10=[$26], $f11=[$27], $f12=[$28], $f13=[$29], $f14=[$30], $f15=[$31], $f16=[$32], $f17=[$33], $f18=[$34], $f19=[$35], $f20=[$36], $f21=[$37], $f22=[$38], $f23=[$39], $f24=[$40], $f25=[$41], $f26=[$42], $f27=[$43], $f28=[$44], $f29=[$45], $f30=[$46], $f31=[$47], $f32=[$48], $f33=[$49], $f34=[$50], $f35=[$51], $f36=[$52], $f37=[$58], $f38=[$59], $f39=[$60], $f40=[$61], $f41=[$62], $f42=[$63], $f43=[$64], $f44=[$65], $f45=[$66], $f46=[$67], $f47=[$68], $f48=[$69], $f49=[$70], $f50=[$71], $f51=[$72], $f52=[$53], $f53=[$54], $f54=[$55], $f55=[$56], $f56=[$57], $f57=[$0], $f58=[$1], $f59=[$2], $f60=[$3], $f61=[$4], $f62=[$5], $f63=[$6], $f64=[$7], $f65=[$8], $f66=[$9], $f67=[$10], $f68=[$11], $f69=[$12], $f70=[$13], $f71=[$14], $f72=[$15], $f73=[$16], $f74=[$17], $f75=[$18], $f76=[$19], $f77=[$20], $f78=[$21], $f79=[$22], $f80=[$23])\n"
-        + "    EnumerableJoinRel(condition=[=($0, $77)], joinType=[inner])\n"
-        + "      EnumerableTableAccessRel(table=[[foodmart2, store]])\n"
-        + "      EnumerableJoinRel(condition=[=($0, $51)], joinType=[inner])\n"
-        + "        EnumerableFilterRel(condition=[=($9, 'San Francisco')])\n"
-        + "          EnumerableTableAccessRel(table=[[foodmart2, customer]])\n"
-        + "        EnumerableJoinRel(condition=[=($6, $20)], joinType=[inner])\n"
-        + "          EnumerableJoinRel(condition=[=($0, $5)], joinType=[inner])\n"
-        + "            EnumerableTableAccessRel(table=[[foodmart2, product_class]])\n"
-        + "            EnumerableTableAccessRel(table=[[foodmart2, product]])\n"
-        + "          EnumerableTableAccessRel(table=[[foodmart2, sales_fact_1997]])\n");
+            + "from \"sales_fact_1997\" as s\n"
+            + "  join \"customer\" as c using (\"customer_id\")\n"
+            + "  join \"product\" as p using (\"product_id\")\n"
+            + "  join \"product_class\" as pc using (\"product_class_id\")\n"
+            + "  join \"store\" as st using (\"store_id\")\n"
+            + "where c.\"city\" = 'San Francisco'\n",
+        "EnumerableProject(product_id=[$0], time_id=[$1], customer_id=[$2], promotion_id=[$3], store_id=[$4], store_sales=[$5], store_cost=[$6], unit_sales=[$7], customer_id0=[$8], account_num=[$9], lname=[$10], fname=[$11], mi=[$12], address1=[$13], address2=[$14], address3=[$15], address4=[$16], city=[$17], state_province=[$18], postal_code=[$19], country=[$20], customer_region_id=[$21], phone1=[$22], phone2=[$23], birthdate=[$24], marital_status=[$25], yearly_income=[$26], gender=[$27], total_children=[$28], num_children_at_home=[$29], education=[$30], date_accnt_opened=[$31], member_card=[$32], occupation=[$33], houseowner=[$34], num_cars_owned=[$35], fullname=[$36], product_class_id=[$37], product_id0=[$38], brand_name=[$39], product_name=[$40], SKU=[$41], SRP=[$42], gross_weight=[$43], net_weight=[$44], recyclable_package=[$45], low_fat=[$46], units_per_case=[$47], cases_per_pallet=[$48], shelf_width=[$49], shelf_height=[$50], shelf_depth=[$51], product_class_id0=[$52], produc
 t_subcategory=[$53], product_category=[$54], product_department=[$55], product_family=[$56], store_id0=[$57], store_type=[$58], region_id=[$59], store_name=[$60], store_number=[$61], store_street_address=[$62], store_city=[$63], store_state=[$64], store_postal_code=[$65], store_country=[$66], store_manager=[$67], store_phone=[$68], store_fax=[$69], first_opened_date=[$70], last_remodel_date=[$71], store_sqft=[$72], grocery_sqft=[$73], frozen_sqft=[$74], meat_sqft=[$75], coffee_bar=[$76], video_store=[$77], salad_bar=[$78], prepared_food=[$79], florist=[$80])\n"
+            + "  EnumerableProject($f0=[$73], $f1=[$74], $f2=[$75], $f3=[$76], $f4=[$77], $f5=[$78], $f6=[$79], $f7=[$80], $f8=[$24], $f9=[$25], $f10=[$26], $f11=[$27], $f12=[$28], $f13=[$29], $f14=[$30], $f15=[$31], $f16=[$32], $f17=[$33], $f18=[$34], $f19=[$35], $f20=[$36], $f21=[$37], $f22=[$38], $f23=[$39], $f24=[$40], $f25=[$41], $f26=[$42], $f27=[$43], $f28=[$44], $f29=[$45], $f30=[$46], $f31=[$47], $f32=[$48], $f33=[$49], $f34=[$50], $f35=[$51], $f36=[$52], $f37=[$58], $f38=[$59], $f39=[$60], $f40=[$61], $f41=[$62], $f42=[$63], $f43=[$64], $f44=[$65], $f45=[$66], $f46=[$67], $f47=[$68], $f48=[$69], $f49=[$70], $f50=[$71], $f51=[$72], $f52=[$53], $f53=[$54], $f54=[$55], $f55=[$56], $f56=[$57], $f57=[$0], $f58=[$1], $f59=[$2], $f60=[$3], $f61=[$4], $f62=[$5], $f63=[$6], $f64=[$7], $f65=[$8], $f66=[$9], $f67=[$10], $f68=[$11], $f69=[$12], $f70=[$13], $f71=[$14], $f72=[$15], $f73=[$16], $f74=[$17], $f75=[$18], $f76=[$19], $f77=[$20], $f78=[$21], $f79=[$22], $f80=[$23])\n"
+            + "    EnumerableJoin(condition=[=($0, $77)], joinType=[inner])\n"
+            + "      EnumerableTableScan(table=[[foodmart2, store]])\n"
+            + "      EnumerableJoin(condition=[=($0, $51)], joinType=[inner])\n"
+            + "        EnumerableFilter(condition=[=($9, 'San Francisco')])\n"
+            + "          EnumerableTableScan(table=[[foodmart2, customer]])\n"
+            + "        EnumerableJoin(condition=[=($6, $20)], joinType=[inner])\n"
+            + "          EnumerableJoin(condition=[=($0, $5)], joinType=[inner])\n"
+            + "            EnumerableTableScan(table=[[foodmart2, product_class]])\n"
+            + "            EnumerableTableScan(table=[[foodmart2, product]])\n"
+            + "          EnumerableTableScan(table=[[foodmart2, sales_fact_1997]])\n");
   }
 
   /** Tests the bushy join algorithm where one table does not join to
    * anything. */
   @Test public void testBushyCrossJoin() throws Exception {
     checkBushy("select * from \"sales_fact_1997\"\n"
-        + "join \"customer\" using (\"customer_id\")\n"
-        + "cross join \"department\"",
-        "EnumerableProjectRel(product_id=[$0], time_id=[$1], customer_id=[$2], promotion_id=[$3], store_id=[$4], store_sales=[$5], store_cost=[$6], unit_sales=[$7], customer_id0=[$8], account_num=[$9], lname=[$10], fname=[$11], mi=[$12], address1=[$13], address2=[$14], address3=[$15], address4=[$16], city=[$17], state_province=[$18], postal_code=[$19], country=[$20], customer_region_id=[$21], phone1=[$22], phone2=[$23], birthdate=[$24], marital_status=[$25], yearly_income=[$26], gender=[$27], total_children=[$28], num_children_at_home=[$29], education=[$30], date_accnt_opened=[$31], member_card=[$32], occupation=[$33], houseowner=[$34], num_cars_owned=[$35], fullname=[$36], department_id=[$37], department_description=[$38])\n"
-        + "  EnumerableProjectRel($f0=[$31], $f1=[$32], $f2=[$33], $f3=[$34], $f4=[$35], $f5=[$36], $f6=[$37], $f7=[$38], $f8=[$2], $f9=[$3], $f10=[$4], $f11=[$5], $f12=[$6], $f13=[$7], $f14=[$8], $f15=[$9], $f16=[$10], $f17=[$11], $f18=[$12], $f19=[$13], $f20=[$14], $f21=[$15], $f22=[$16], $f23=[$17], $f24=[$18], $f25=[$19], $f26=[$20], $f27=[$21], $f28=[$22], $f29=[$23], $f30=[$24], $f31=[$25], $f32=[$26], $f33=[$27], $f34=[$28], $f35=[$29], $f36=[$30], $f37=[$0], $f38=[$1])\n"
-        + "    EnumerableJoinRel(condition=[true], joinType=[inner])\n"
-        + "      EnumerableTableAccessRel(table=[[foodmart2, department]])\n"
-        + "      EnumerableJoinRel(condition=[=($0, $31)], joinType=[inner])\n"
-        + "        EnumerableTableAccessRel(table=[[foodmart2, customer]])\n"
-        + "        EnumerableTableAccessRel(table=[[foodmart2, sales_fact_1997]])");
+            + "join \"customer\" using (\"customer_id\")\n"
+            + "cross join \"department\"",
+        "EnumerableProject(product_id=[$0], time_id=[$1], customer_id=[$2], promotion_id=[$3], store_id=[$4], store_sales=[$5], store_cost=[$6], unit_sales=[$7], customer_id0=[$8], account_num=[$9], lname=[$10], fname=[$11], mi=[$12], address1=[$13], address2=[$14], address3=[$15], address4=[$16], city=[$17], state_province=[$18], postal_code=[$19], country=[$20], customer_region_id=[$21], phone1=[$22], phone2=[$23], birthdate=[$24], marital_status=[$25], yearly_income=[$26], gender=[$27], total_children=[$28], num_children_at_home=[$29], education=[$30], date_accnt_opened=[$31], member_card=[$32], occupation=[$33], houseowner=[$34], num_cars_owned=[$35], fullname=[$36], department_id=[$37], department_description=[$38])\n"
+            + "  EnumerableProject($f0=[$31], $f1=[$32], $f2=[$33], $f3=[$34], $f4=[$35], $f5=[$36], $f6=[$37], $f7=[$38], $f8=[$2], $f9=[$3], $f10=[$4], $f11=[$5], $f12=[$6], $f13=[$7], $f14=[$8], $f15=[$9], $f16=[$10], $f17=[$11], $f18=[$12], $f19=[$13], $f20=[$14], $f21=[$15], $f22=[$16], $f23=[$17], $f24=[$18], $f25=[$19], $f26=[$20], $f27=[$21], $f28=[$22], $f29=[$23], $f30=[$24], $f31=[$25], $f32=[$26], $f33=[$27], $f34=[$28], $f35=[$29], $f36=[$30], $f37=[$0], $f38=[$1])\n"
+            + "    EnumerableJoin(condition=[true], joinType=[inner])\n"
+            + "      EnumerableTableScan(table=[[foodmart2, department]])\n"
+            + "      EnumerableJoin(condition=[=($0, $31)], joinType=[inner])\n"
+            + "        EnumerableTableScan(table=[[foodmart2, customer]])\n"
+            + "        EnumerableTableScan(table=[[foodmart2, sales_fact_1997]])");
   }
 
   /** Tests the bushy join algorithm against a query where not all tables have a
    * join condition to the others. */
   @Test public void testBushyCrossJoin2() throws Exception {
     checkBushy("select * from \"sales_fact_1997\"\n"
-        + "join \"customer\" using (\"customer_id\")\n"
-        + "cross join \"department\"\n"
-        + "join \"employee\" using (\"department_id\")",
-        "EnumerableProjectRel(product_id=[$0], time_id=[$1], customer_id=[$2], promotion_id=[$3], store_id=[$4], store_sales=[$5], store_cost=[$6], unit_sales=[$7], customer_id0=[$8], account_num=[$9], lname=[$10], fname=[$11], mi=[$12], address1=[$13], address2=[$14], address3=[$15], address4=[$16], city=[$17], state_province=[$18], postal_code=[$19], country=[$20], customer_region_id=[$21], phone1=[$22], phone2=[$23], birthdate=[$24], marital_status=[$25], yearly_income=[$26], gender=[$27], total_children=[$28], num_children_at_home=[$29], education=[$30], date_accnt_opened=[$31], member_card=[$32], occupation=[$33], houseowner=[$34], num_cars_owned=[$35], fullname=[$36], department_id=[$37], department_description=[$38], employee_id=[$39], full_name=[$40], first_name=[$41], last_name=[$42], position_id=[$43], position_title=[$44], store_id0=[$45], department_id0=[$46], birth_date=[$47], hire_date=[$48], end_date=[$49], salary=[$50], supervisor_id=[$51], education_level=[$52], mar
 ital_status0=[$53], gender0=[$54], management_role=[$55])\n"
-        + "  EnumerableProjectRel($f0=[$48], $f1=[$49], $f2=[$50], $f3=[$51], $f4=[$52], $f5=[$53], $f6=[$54], $f7=[$55], $f8=[$19], $f9=[$20], $f10=[$21], $f11=[$22], $f12=[$23], $f13=[$24], $f14=[$25], $f15=[$26], $f16=[$27], $f17=[$28], $f18=[$29], $f19=[$30], $f20=[$31], $f21=[$32], $f22=[$33], $f23=[$34], $f24=[$35], $f25=[$36], $f26=[$37], $f27=[$38], $f28=[$39], $f29=[$40], $f30=[$41], $f31=[$42], $f32=[$43], $f33=[$44], $f34=[$45], $f35=[$46], $f36=[$47], $f37=[$0], $f38=[$1], $f39=[$2], $f40=[$3], $f41=[$4], $f42=[$5], $f43=[$6], $f44=[$7], $f45=[$8], $f46=[$9], $f47=[$10], $f48=[$11], $f49=[$12], $f50=[$13], $f51=[$14], $f52=[$15], $f53=[$16], $f54=[$17], $f55=[$18])\n"
-        + "    EnumerableJoinRel(condition=[true], joinType=[inner])\n"
-        + "      EnumerableJoinRel(condition=[=($0, $9)], joinType=[inner])\n"
-        + "        EnumerableTableAccessRel(table=[[foodmart2, department]])\n"
-        + "        EnumerableTableAccessRel(table=[[foodmart2, employee]])\n"
-        + "      EnumerableJoinRel(condition=[=($0, $31)], joinType=[inner])\n"
-        + "        EnumerableTableAccessRel(table=[[foodmart2, customer]])\n"
-        + "        EnumerableTableAccessRel(table=[[foodmart2, sales_fact_1997]])\n");
+            + "join \"customer\" using (\"customer_id\")\n"
+            + "cross join \"department\"\n"
+            + "join \"employee\" using (\"department_id\")",
+        "EnumerableProject(product_id=[$0], time_id=[$1], customer_id=[$2], promotion_id=[$3], store_id=[$4], store_sales=[$5], store_cost=[$6], unit_sales=[$7], customer_id0=[$8], account_num=[$9], lname=[$10], fname=[$11], mi=[$12], address1=[$13], address2=[$14], address3=[$15], address4=[$16], city=[$17], state_province=[$18], postal_code=[$19], country=[$20], customer_region_id=[$21], phone1=[$22], phone2=[$23], birthdate=[$24], marital_status=[$25], yearly_income=[$26], gender=[$27], total_children=[$28], num_children_at_home=[$29], education=[$30], date_accnt_opened=[$31], member_card=[$32], occupation=[$33], houseowner=[$34], num_cars_owned=[$35], fullname=[$36], department_id=[$37], department_description=[$38], employee_id=[$39], full_name=[$40], first_name=[$41], last_name=[$42], position_id=[$43], position_title=[$44], store_id0=[$45], department_id0=[$46], birth_date=[$47], hire_date=[$48], end_date=[$49], salary=[$50], supervisor_id=[$51], education_level=[$52], marita
 l_status0=[$53], gender0=[$54], management_role=[$55])\n"
+            + "  EnumerableProject($f0=[$48], $f1=[$49], $f2=[$50], $f3=[$51], $f4=[$52], $f5=[$53], $f6=[$54], $f7=[$55], $f8=[$19], $f9=[$20], $f10=[$21], $f11=[$22], $f12=[$23], $f13=[$24], $f14=[$25], $f15=[$26], $f16=[$27], $f17=[$28], $f18=[$29], $f19=[$30], $f20=[$31], $f21=[$32], $f22=[$33], $f23=[$34], $f24=[$35], $f25=[$36], $f26=[$37], $f27=[$38], $f28=[$39], $f29=[$40], $f30=[$41], $f31=[$42], $f32=[$43], $f33=[$44], $f34=[$45], $f35=[$46], $f36=[$47], $f37=[$0], $f38=[$1], $f39=[$2], $f40=[$3], $f41=[$4], $f42=[$5], $f43=[$6], $f44=[$7], $f45=[$8], $f46=[$9], $f47=[$10], $f48=[$11], $f49=[$12], $f50=[$13], $f51=[$14], $f52=[$15], $f53=[$16], $f54=[$17], $f55=[$18])\n"
+            + "    EnumerableJoin(condition=[true], joinType=[inner])\n"
+            + "      EnumerableJoin(condition=[=($0, $9)], joinType=[inner])\n"
+            + "        EnumerableTableScan(table=[[foodmart2, department]])\n"
+            + "        EnumerableTableScan(table=[[foodmart2, employee]])\n"
+            + "      EnumerableJoin(condition=[=($0, $31)], joinType=[inner])\n"
+            + "        EnumerableTableScan(table=[[foodmart2, customer]])\n"
+            + "        EnumerableTableScan(table=[[foodmart2, sales_fact_1997]])\n");
   }
 
   /** Checks that a query returns a particular plan, using a planner with
-   * OptimizeBushyJoinRule enabled. */
+   * MultiJoinOptimizeBushyRule enabled. */
   private void checkBushy(String sql, String expected) throws Exception {
     final SchemaPlus rootSchema = Frameworks.createRootSchema(true);
     final FrameworkConfig config = Frameworks.newConfigBuilder()
         .lex(Lex.ORACLE)
         .defaultSchema(
-            OptiqAssert.addSchema(rootSchema,
-                OptiqAssert.SchemaSpec.CLONE_FOODMART))
+            CalciteAssert.addSchema(rootSchema,
+                CalciteAssert.SchemaSpec.CLONE_FOODMART))
         .traitDefs((List<RelTraitDef>) null)
         .programs(Programs.heuristicJoinOrder(Programs.RULE_SET, true, 2))
         .build();
@@ -710,42 +739,44 @@ public class PlannerTest {
   }
 
   /**
-   * Rule to convert a {@link EnumerableProjectRel} to an
-   * {@link JdbcProjectRel}.
+   * Rule to convert a
+   * {@link org.apache.calcite.adapter.enumerable.EnumerableProject} to an
+   * {@link org.apache.calcite.adapter.jdbc.JdbcRules.JdbcProject}.
    */
   private class MockJdbcProjectRule extends ConverterRule {
     private MockJdbcProjectRule(JdbcConvention out) {
-      super(EnumerableProjectRel.class, EnumerableConvention.INSTANCE, out,
+      super(EnumerableProject.class, EnumerableConvention.INSTANCE, out,
           "MockJdbcProjectRule");
     }
 
     public RelNode convert(RelNode rel) {
-      final EnumerableProjectRel project = (EnumerableProjectRel) rel;
+      final EnumerableProject project = (EnumerableProject) rel;
 
-      return new JdbcProjectRel(
+      return new JdbcRules.JdbcProject(
           rel.getCluster(),
           rel.getTraitSet().replace(getOutConvention()),
-          convert(project.getChild(),
-              project.getChild().getTraitSet().replace(getOutConvention())),
+          convert(project.getInput(),
+              project.getInput().getTraitSet().replace(getOutConvention())),
           project.getProjects(),
           project.getRowType(),
-          ProjectRelBase.Flags.BOXED);
+          Project.Flags.BOXED);
     }
   }
 
   /**
-   * Rule to convert a {@link JavaRules.EnumerableTableAccessRel} to an
+   * Rule to convert a
+   * {@link org.apache.calcite.adapter.enumerable.EnumerableTableScan} to an
    * {@link MockJdbcTableScan}.
    */
   private class MockJdbcTableRule extends ConverterRule {
     private MockJdbcTableRule(JdbcConvention out) {
-      super(JavaRules.EnumerableTableAccessRel.class,
+      super(EnumerableTableScan.class,
           EnumerableConvention.INSTANCE, out, "MockJdbcTableRule");
     }
 
     public RelNode convert(RelNode rel) {
-      final JavaRules.EnumerableTableAccessRel scan =
-          (JavaRules.EnumerableTableAccessRel) rel;
+      final EnumerableTableScan scan =
+          (EnumerableTableScan) rel;
       return new MockJdbcTableScan(scan.getCluster(),
           scan.getTable(),
           (JdbcConvention) getOutConvention());
@@ -756,7 +787,7 @@ public class PlannerTest {
    * Relational expression representing a "mock" scan of a table in a
    * JDBC data source.
    */
-  private class MockJdbcTableScan extends TableAccessRelBase
+  private class MockJdbcTableScan extends TableScan
       implements JdbcRel {
 
     public MockJdbcTableScan(RelOptCluster cluster, RelOptTable table,
@@ -782,7 +813,7 @@ public class PlannerTest {
   }
 
   /**
-   * Test to determine whether de-correlation correctly removes CorrelatorRel.
+   * Test to determine whether de-correlation correctly removes Correlator.
    */
   @Test public void testOldJoinStyleDeCorrelation() throws Exception {
     assertFalse(
@@ -800,7 +831,7 @@ public class PlannerTest {
             + "    where\n"
             + "      p.`pPartkey` = ps.`psPartkey`\n"
             + "  )\n")
-            .contains("CorrelatorRel"));
+            .contains("Correlator"));
   }
 
   public String checkTpchQuery(String tpchTestQuery) throws Exception {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/tools/TpchSchema.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/tools/TpchSchema.java b/core/src/test/java/org/apache/calcite/tools/TpchSchema.java
index 5e73e88..be02229 100644
--- a/core/src/test/java/org/apache/calcite/tools/TpchSchema.java
+++ b/core/src/test/java/org/apache/calcite/tools/TpchSchema.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.tools;
+package org.apache.calcite.tools;
 
 /**
  * TPC-H table schema.
@@ -35,8 +35,7 @@ public class TpchSchema {
       this.pPartkey = pPartkey;
     }
 
-    @Override
-    public String toString() {
+    @Override public String toString() {
       return "Part [pPartkey=" + pPartkey + "]";
     }
   }
@@ -54,8 +53,7 @@ public class TpchSchema {
       this.psSupplyCost = psSupplyCost;
     }
 
-    @Override
-    public String toString() {
+    @Override public String toString() {
       return "PartSupp [pSupplyCost=" + psPartkey + ", pSupplyCost="
         + psSupplyCost + "]";
     }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/util/BitSetsTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/util/BitSetsTest.java b/core/src/test/java/org/apache/calcite/util/BitSetsTest.java
index 88a6ea9..4e7cdb9 100644
--- a/core/src/test/java/org/apache/calcite/util/BitSetsTest.java
+++ b/core/src/test/java/org/apache/calcite/util/BitSetsTest.java
@@ -14,9 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.util;
-
-import org.eigenbase.util.ImmutableIntList;
+package org.apache.calcite.util;
 
 import com.google.common.collect.Maps;
 
@@ -35,12 +33,12 @@ import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 /**
- * Unit test for {@link net.hydromatic.optiq.util.BitSets}.
+ * Unit test for {@link org.apache.calcite.util.BitSets}.
  */
 public class BitSetsTest {
   /**
    * Tests the method
-   * {@link net.hydromatic.optiq.util.BitSets#toIter(java.util.BitSet)}.
+   * {@link org.apache.calcite.util.BitSets#toIter(java.util.BitSet)}.
    */
   @Test public void testToIterBitSet() {
     BitSet bitSet = new BitSet();
@@ -74,7 +72,7 @@ public class BitSetsTest {
 
   /**
    * Tests the method
-   * {@link net.hydromatic.optiq.util.BitSets#toList(java.util.BitSet)}.
+   * {@link org.apache.calcite.util.BitSets#toList(java.util.BitSet)}.
    */
   @Test public void testToListBitSet() {
     BitSet bitSet = new BitSet(10);
@@ -86,7 +84,7 @@ public class BitSetsTest {
   }
 
   /**
-   * Tests the method {@link net.hydromatic.optiq.util.BitSets#of(int...)}.
+   * Tests the method {@link org.apache.calcite.util.BitSets#of(int...)}.
    */
   @Test public void testBitSetOf() {
     assertEquals(
@@ -98,7 +96,7 @@ public class BitSetsTest {
   }
 
   /**
-   * Tests the method {@link net.hydromatic.optiq.util.BitSets#range(int, int)}.
+   * Tests the method {@link org.apache.calcite.util.BitSets#range(int, int)}.
    */
   @Test public void testBitSetsRange() {
     assertEquals(
@@ -114,7 +112,7 @@ public class BitSetsTest {
 
   /**
    * Tests the method
-   * {@link net.hydromatic.optiq.util.BitSets#toArray(java.util.BitSet)}.
+   * {@link org.apache.calcite.util.BitSets#toArray(java.util.BitSet)}.
    */
   @Test public void testBitSetsToArray() {
     int[][] arrays = {{}, {0}, {0, 2}, {1, 65}, {100}};
@@ -125,7 +123,7 @@ public class BitSetsTest {
 
   /**
    * Tests the method
-   * {@link net.hydromatic.optiq.util.BitSets#union(java.util.BitSet, java.util.BitSet...)}.
+   * {@link org.apache.calcite.util.BitSets#union(java.util.BitSet, java.util.BitSet...)}.
    */
   @Test public void testBitSetsUnion() {
     assertThat(BitSets.union(BitSets.of(1), BitSets.of(3)).toString(),
@@ -140,7 +138,7 @@ public class BitSetsTest {
 
   /**
    * Tests the method
-   * {@link net.hydromatic.optiq.util.BitSets#contains(java.util.BitSet, java.util.BitSet)}.
+   * {@link org.apache.calcite.util.BitSets#contains(java.util.BitSet, java.util.BitSet)}.
    */
   @Test public void testBitSetsContains() {
     assertTrue(BitSets.contains(BitSets.range(0, 5), BitSets.range(2, 4)));
@@ -158,7 +156,7 @@ public class BitSetsTest {
 
   /**
    * Tests the method
-   * {@link net.hydromatic.optiq.util.BitSets#of(org.eigenbase.util.ImmutableIntList)}.
+   * {@link org.apache.calcite.util.BitSets#of(ImmutableIntList)}.
    */
   @Test public void testBitSetOfImmutableIntList() {
     ImmutableIntList list = ImmutableIntList.of();
@@ -170,7 +168,7 @@ public class BitSetsTest {
 
   /**
    * Tests the method
-   * {@link net.hydromatic.optiq.util.BitSets#previousClearBit(java.util.BitSet, int)}.
+   * {@link org.apache.calcite.util.BitSets#previousClearBit(java.util.BitSet, int)}.
    */
   @Test public void testPreviousClearBit() {
     assertThat(BitSets.previousClearBit(BitSets.of(), 10), equalTo(10));

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/util/ChunkListTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/util/ChunkListTest.java b/core/src/test/java/org/apache/calcite/util/ChunkListTest.java
index 6dfb5a7..0e2bb2d 100644
--- a/core/src/test/java/org/apache/calcite/util/ChunkListTest.java
+++ b/core/src/test/java/org/apache/calcite/util/ChunkListTest.java
@@ -14,16 +14,27 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.util;
+package org.apache.calcite.util;
 
-import java.util.*;
-
-import net.hydromatic.linq4j.function.Function0;
-import net.hydromatic.linq4j.function.Function1;
+import org.apache.calcite.linq4j.function.Function0;
+import org.apache.calcite.linq4j.function.Function1;
 
 import org.junit.Test;
 
-import static org.junit.Assert.*;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.Random;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 /**
  * Unit and performance test for {@link ChunkList}.
@@ -312,9 +323,8 @@ public class ChunkListTest {
         if (size.left > 1000000) {
           continue;
         }
-        new Benchmark(
-            "get from " + size.right + " values, " + (size.left / 1000)
-            + " times, " + pair.right,
+        new Benchmark("get from " + size.right + " values, "
+            + (size.left / 1000) + " times, " + pair.right,
             new Function1<Benchmark.Statistician, Void>() {
               public Void apply(Benchmark.Statistician statistician) {
                 final List<Integer> list = pair.left.apply();

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/util/PartiallyOrderedSetTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/util/PartiallyOrderedSetTest.java b/core/src/test/java/org/apache/calcite/util/PartiallyOrderedSetTest.java
index a65d8da..d37c223 100644
--- a/core/src/test/java/org/apache/calcite/util/PartiallyOrderedSetTest.java
+++ b/core/src/test/java/org/apache/calcite/util/PartiallyOrderedSetTest.java
@@ -14,17 +14,22 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.util;
+package org.apache.calcite.util;
 
-import net.hydromatic.optiq.test.OptiqAssert;
-
-import org.eigenbase.util.TestUtil;
+import org.apache.calcite.test.CalciteAssert;
 
 import org.junit.Test;
 
-import java.util.*;
+import java.util.AbstractList;
+import java.util.Collection;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Random;
+import java.util.Set;
+import java.util.TreeSet;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 /**
  * Unit test for {@link PartiallyOrderedSet}.
@@ -33,7 +38,7 @@ public class PartiallyOrderedSetTest {
   private static final boolean DEBUG = false;
 
   // 100, 250, 1000, 3000 are reasonable
-  private static final int SCALE = OptiqAssert.ENABLE_SLOW ? 250 : 50;
+  private static final int SCALE = CalciteAssert.ENABLE_SLOW ? 250 : 50;
 
   final long seed = new Random().nextLong();
   final Random random = new Random(seed);
@@ -97,7 +102,7 @@ public class PartiallyOrderedSetTest {
     poset.out(buf);
     TestUtil.assertEqualsVerbose(
         "PartiallyOrderedSet size: 0 elements: {\n"
-        + "}",
+            + "}",
         buf.toString());
 
     poset.add("a");
@@ -139,11 +144,11 @@ public class PartiallyOrderedSetTest {
     poset.out(buf);
     TestUtil.assertEqualsVerbose(
         "PartiallyOrderedSet size: 4 elements: {\n"
-        + "  'abcd' parents: [] children: ['ab', 'bcd']\n"
-        + "  'ab' parents: ['abcd'] children: ['']\n"
-        + "  'bcd' parents: ['abcd'] children: ['']\n"
-        + "  '' parents: ['ab', 'bcd'] children: []\n"
-        + "}",
+            + "  'abcd' parents: [] children: ['ab', 'bcd']\n"
+            + "  'ab' parents: ['abcd'] children: ['']\n"
+            + "  'bcd' parents: ['abcd'] children: ['']\n"
+            + "  '' parents: ['ab', 'bcd'] children: []\n"
+            + "}",
         buf.toString());
 
     final String b = "'b'";
@@ -218,7 +223,7 @@ public class PartiallyOrderedSetTest {
   }
 
   @Test public void testDivisorPoset() {
-    if (!OptiqAssert.ENABLE_SLOW) {
+    if (!CalciteAssert.ENABLE_SLOW) {
       return;
     }
     PartiallyOrderedSet<Integer> integers =
@@ -376,13 +381,11 @@ public class PartiallyOrderedSetTest {
   private static Collection<Integer> range(
       final int start, final int end) {
     return new AbstractList<Integer>() {
-      @Override
-      public Integer get(int index) {
+      @Override public Integer get(int index) {
         return start + index;
       }
 
-      @Override
-      public int size() {
+      @Override public int size() {
         return end - start;
       }
     };

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/util/PermutationTestCase.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/util/PermutationTestCase.java b/core/src/test/java/org/apache/calcite/util/PermutationTestCase.java
index 5648a6f..04beeb1 100644
--- a/core/src/test/java/org/apache/calcite/util/PermutationTestCase.java
+++ b/core/src/test/java/org/apache/calcite/util/PermutationTestCase.java
@@ -14,12 +14,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.util;
+package org.apache.calcite.util;
 
 
 import org.junit.Test;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 /**
  * Unit test for {@link Permutation}.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/util/ReflectVisitorTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/util/ReflectVisitorTest.java b/core/src/test/java/org/apache/calcite/util/ReflectVisitorTest.java
index 12c0fa8..877f446 100644
--- a/core/src/test/java/org/apache/calcite/util/ReflectVisitorTest.java
+++ b/core/src/test/java/org/apache/calcite/util/ReflectVisitorTest.java
@@ -14,17 +14,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.util;
-
-import java.math.*;
+package org.apache.calcite.util;
 
 import org.junit.Test;
 
-import static org.junit.Assert.*;
+import java.math.BigDecimal;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 /**
- * ReflectVisitorTest tests {@link ReflectUtil#invokeVisitor} and {@link
- * ReflectiveVisitor} and provides a contrived example of how to use them.
+ * ReflectVisitorTest tests {@link ReflectUtil#invokeVisitor} and
+ * {@link ReflectiveVisitor} and provides a contrived example of how to use
+ * them.
  */
 public class ReflectVisitorTest {
   //~ Constructors -----------------------------------------------------------
@@ -138,6 +141,7 @@ public class ReflectVisitorTest {
   public interface FudgeableNumber {
   }
 
+  /** Sub-interface of {@link FudgeableNumber}. */
   public interface DiceyNumber extends FudgeableNumber {
   }
 
@@ -176,8 +180,8 @@ public class ReflectVisitorTest {
 
     /**
      * Negates the given number without using a dispatcher object to cache
-     * applicable methods. The results should be the same as {@link
-     * #negate(Number)}.
+     * applicable methods. The results should be the same as
+     * {@link #negate(Number)}.
      *
      * @param n the number to be negated
      * @return the negated result; not guaranteed to be the same concrete

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/util/TestUtil.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/util/TestUtil.java b/core/src/test/java/org/apache/calcite/util/TestUtil.java
index 34dfd6a..769bf75 100644
--- a/core/src/test/java/org/apache/calcite/util/TestUtil.java
+++ b/core/src/test/java/org/apache/calcite/util/TestUtil.java
@@ -14,12 +14,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.util;
-
-import java.util.regex.*;
+package org.apache.calcite.util;
 
 import org.junit.ComparisonFailure;
 
+import java.util.regex.Pattern;
+
 /**
  * Static utilities for JUnit tests.
  */
@@ -43,9 +43,7 @@ public abstract class TestUtil {
       if (expected == null) {
         return;
       } else {
-        String message =
-            "Expected:\n"
-            + expected
+        String message = "Expected:\n" + expected
             + "\nActual: null";
         throw new ComparisonFailure(message, expected, null);
       }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/util/UtilTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/util/UtilTest.java b/core/src/test/java/org/apache/calcite/util/UtilTest.java
index b6bcf00..4ac9cff 100644
--- a/core/src/test/java/org/apache/calcite/util/UtilTest.java
+++ b/core/src/test/java/org/apache/calcite/util/UtilTest.java
@@ -14,29 +14,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.util;
+package org.apache.calcite.util;
 
-import java.io.*;
-import java.lang.management.*;
-import java.math.*;
-import java.sql.Timestamp;
-import java.text.MessageFormat;
-import java.util.*;
-
-import javax.annotation.Nullable;
-
-import org.eigenbase.resource.Resources;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.util.*;
-import org.eigenbase.test.*;
-
-import net.hydromatic.linq4j.function.Function1;
-
-import net.hydromatic.optiq.runtime.FlatLists;
-import net.hydromatic.optiq.runtime.Spaces;
-import net.hydromatic.optiq.util.BitSets;
-import net.hydromatic.optiq.util.Compatible;
-import net.hydromatic.optiq.util.CompositeMap;
+import org.apache.calcite.linq4j.function.Function1;
+import org.apache.calcite.runtime.FlatLists;
+import org.apache.calcite.runtime.Resources;
+import org.apache.calcite.runtime.Spaces;
+import org.apache.calcite.sql.SqlDialect;
+import org.apache.calcite.sql.util.SqlBuilder;
+import org.apache.calcite.sql.util.SqlString;
+import org.apache.calcite.test.DiffTestCase;
 
 import com.google.common.base.Function;
 import com.google.common.collect.ImmutableList;
@@ -48,8 +35,43 @@ import com.google.common.collect.Lists;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-import static org.hamcrest.CoreMatchers.*;
-import static org.junit.Assert.*;
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.io.UnsupportedEncodingException;
+import java.lang.management.MemoryType;
+import java.math.BigDecimal;
+import java.sql.Timestamp;
+import java.text.MessageFormat;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.BitSet;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.NavigableSet;
+import java.util.Properties;
+import java.util.Random;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TimeZone;
+import java.util.TreeSet;
+import javax.annotation.Nullable;
+
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.CoreMatchers.isA;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 /**
  * Unit test for {@link Util} and other classes in this package.
@@ -316,7 +338,7 @@ public class UtilTest {
   }
 
   /**
-   * Tests {@link CastingList} and {@link Util#cast}.
+   * Tests {@link org.apache.calcite.util.CastingList} and {@link Util#cast}.
    */
   @Test public void testCastingList() {
     final List<Number> numberList = new ArrayList<Number>();
@@ -398,16 +420,15 @@ public class UtilTest {
             Arrays.asList(before),
             Arrays.asList(after));
     assertThat(Util.toLinux(diff),
-        equalTo(
-            "1a2\n"
-                + "> (they call her \"Polythene Pam\")\n"
-                + "3c4,5\n"
-                + "< She's the kind of a girl that makes The News of The World\n"
-                + "---\n"
-                + "> She's the kind of a girl that makes The Sunday Times\n"
-                + "> seem more interesting.\n"
-                + "5d6\n"
-                + "< Yeah yeah yeah.\n"));
+        equalTo("1a2\n"
+            + "> (they call her \"Polythene Pam\")\n"
+            + "3c4,5\n"
+            + "< She's the kind of a girl that makes The News of The World\n"
+            + "---\n"
+            + "> She's the kind of a girl that makes The Sunday Times\n"
+            + "> seem more interesting.\n"
+            + "5d6\n"
+            + "< Yeah yeah yeah.\n"));
   }
 
   /**
@@ -469,8 +490,8 @@ public class UtilTest {
   }
 
   /**
-   * Tests the methods {@link Util#enumConstants(Class)} and {@link
-   * Util#enumVal(Class, String)}.
+   * Tests the methods {@link Util#enumConstants(Class)} and
+   * {@link Util#enumVal(Class, String)}.
    */
   @Test public void testEnumConstants() {
     final Map<String, MemoryType> memoryTypeMap =
@@ -494,7 +515,7 @@ public class UtilTest {
    * Tests SQL builders.
    */
   @Test public void testSqlBuilder() {
-    final SqlBuilder buf = new SqlBuilder(SqlDialect.EIGENBASE);
+    final SqlBuilder buf = new SqlBuilder(SqlDialect.CALCITE);
     assertEquals(0, buf.length());
     buf.append("select ");
     assertEquals("select ", buf.getSql());
@@ -507,7 +528,7 @@ public class UtilTest {
     assertEquals("select \"x\", \"y\".\"a b\"", buf.getSql());
 
     final SqlString sqlString = buf.toSqlString();
-    assertEquals(SqlDialect.EIGENBASE, sqlString.getDialect());
+    assertEquals(SqlDialect.CALCITE, sqlString.getDialect());
     assertEquals(buf.getSql(), sqlString.getSql());
 
     assertTrue(buf.getSql().length() > 0);
@@ -533,7 +554,7 @@ public class UtilTest {
   }
 
   /**
-   * Unit test for {@link org.eigenbase.util.CompositeList}.
+   * Unit test for {@link org.apache.calcite.util.CompositeList}.
    */
   @Test public void testCompositeList() {
     // Made up of zero lists
@@ -923,30 +944,6 @@ public class UtilTest {
   }
 
   /**
-   * Unit test for {@link Util#toCamelCase(String)}.
-   */
-  @Test public void testToCamelCase() {
-    assertEquals("myJdbcDriver", Util.toCamelCase("MY_JDBC_DRIVER"));
-    assertEquals("myJdbcDriver", Util.toCamelCase("MY_JDBC__DRIVER"));
-    assertEquals("myJdbcDriver", Util.toCamelCase("my_jdbc_driver"));
-    assertEquals("abCdefGHij", Util.toCamelCase("ab_cdEf_g_Hij"));
-    assertEquals("JdbcDriver", Util.toCamelCase("_JDBC_DRIVER"));
-    assertEquals("", Util.toCamelCase("_"));
-    assertEquals("", Util.toCamelCase(""));
-  }
-
-  /**
-   * Unit test for {@link Util#camelToUpper(String)}.
-   */
-  @Test public void testCamelToUpper() {
-    assertEquals("MY_JDBC_DRIVER", Util.camelToUpper("myJdbcDriver"));
-    assertEquals("MY_J_D_B_C_DRIVER", Util.camelToUpper("myJDBCDriver"));
-    assertEquals("AB_CDEF_G_HIJ", Util.camelToUpper("abCdefGHij"));
-    assertEquals("_JDBC_DRIVER", Util.camelToUpper("JdbcDriver"));
-    assertEquals("", Util.camelToUpper(""));
-  }
-
-  /**
    * Unit test for {@link Util#isDistinct(java.util.List)}.
    */
   @Test public void testDistinct() {
@@ -959,7 +956,7 @@ public class UtilTest {
   }
 
   /**
-   * Unit test for {@link org.eigenbase.util.JsonBuilder}.
+   * Unit test for {@link org.apache.calcite.util.JsonBuilder}.
    */
   @Test public void testJsonBuilder() {
     JsonBuilder builder = new JsonBuilder();
@@ -1196,7 +1193,7 @@ public class UtilTest {
     return set.subSet(s.toUpperCase(), true, s.toLowerCase(), true);
   }
 
-  /** Test for {@link org.eigenbase.util.ImmutableNullableList}. */
+  /** Test for {@link org.apache.calcite.util.ImmutableNullableList}. */
   @Test public void testImmutableNullableList() {
     final List<String> arrayList = Arrays.asList("a", null, "c");
     final List<String> list = ImmutableNullableList.copyOf(arrayList);
@@ -1237,7 +1234,7 @@ public class UtilTest {
         isA((Class) ImmutableList.class));
   }
 
-  /** Test for {@link org.eigenbase.util.UnmodifiableArrayList}. */
+  /** Test for {@link org.apache.calcite.util.UnmodifiableArrayList}. */
   @Test public void testUnmodifiableArrayList() {
     final String[] strings = {"a", null, "c"};
     final List<String> arrayList = Arrays.asList(strings);

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/util/graph/DirectedGraphTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/util/graph/DirectedGraphTest.java b/core/src/test/java/org/apache/calcite/util/graph/DirectedGraphTest.java
index 6ff7d35..6724ffc 100644
--- a/core/src/test/java/org/apache/calcite/util/graph/DirectedGraphTest.java
+++ b/core/src/test/java/org/apache/calcite/util/graph/DirectedGraphTest.java
@@ -14,20 +14,29 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.util.graph;
+package org.apache.calcite.util.graph;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Lists;
 
 import org.hamcrest.CoreMatchers;
-
 import org.junit.Test;
 
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Set;
 
 import static org.hamcrest.CoreMatchers.equalTo;
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 /**
  * Unit test for {@link DirectedGraph}.
@@ -172,7 +181,7 @@ public class DirectedGraphTest {
   }
 
   /** Unit test for
-   * {@link net.hydromatic.optiq.util.graph.Graphs.FrozenGraph}. */
+   * {@link org.apache.calcite.util.graph.Graphs.FrozenGraph}. */
   @Test public void testPaths() {
     //       B -> C
     //      /      \
@@ -205,7 +214,7 @@ public class DirectedGraphTest {
     assertEquals("[D, E]", frozenGraph.getShortestPath("D", "E").toString());
   }
 
-  /** Unit test for {@link net.hydromatic.optiq.util.graph.CycleDetector}. */
+  /** Unit test for {@link org.apache.calcite.util.graph.CycleDetector}. */
   @Test public void testCycleDetection() {
     // A - B - C - D
     //  \     /
@@ -276,7 +285,7 @@ public class DirectedGraphTest {
   }
 
   /** Unit test for
-   * {@link net.hydromatic.optiq.util.graph.BreadthFirstIterator}. */
+   * {@link org.apache.calcite.util.graph.BreadthFirstIterator}. */
   @Test public void testBreadthFirstIterator() {
     DefaultDirectedGraph<String, DefaultEdge> graph = createDag();
     final List<String> expected =

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/util/mapping/MappingTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/util/mapping/MappingTest.java b/core/src/test/java/org/apache/calcite/util/mapping/MappingTest.java
index 65a9d61..2155294 100644
--- a/core/src/test/java/org/apache/calcite/util/mapping/MappingTest.java
+++ b/core/src/test/java/org/apache/calcite/util/mapping/MappingTest.java
@@ -14,18 +14,22 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.util.mapping;
-
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
+package org.apache.calcite.util.mapping;
 
 import com.google.common.collect.ImmutableMap;
 
 import org.junit.Test;
 
-import static org.hamcrest.CoreMatchers.*;
-import static org.junit.Assert.*;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 /**
  * Unit test for mappings.


[53/58] [abbrv] incubator-calcite git commit: [CALCITE-460] Add ImmutableBitSet and replace uses of BitSet

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/core/src/main/java/org/apache/calcite/rel/rules/MultiJoin.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/MultiJoin.java b/core/src/main/java/org/apache/calcite/rel/rules/MultiJoin.java
index cd70844..ce651bc 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/MultiJoin.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/MultiJoin.java
@@ -26,6 +26,7 @@ import org.apache.calcite.rel.RelWriter;
 import org.apache.calcite.rel.core.JoinRelType;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.ImmutableIntList;
 import org.apache.calcite.util.ImmutableNullableList;
 
@@ -34,7 +35,6 @@ import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Lists;
 
 import java.util.ArrayList;
-import java.util.BitSet;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -52,7 +52,7 @@ public final class MultiJoin extends AbstractRelNode {
   private final boolean isFullOuterJoin;
   private final List<RexNode> outerJoinConditions;
   private final ImmutableList<JoinRelType> joinTypes;
-  private final List<BitSet> projFields;
+  private final List<ImmutableBitSet> projFields;
   public final ImmutableMap<Integer, ImmutableIntList> joinFieldRefCountsMap;
   private final RexNode postJoinFilter;
 
@@ -91,7 +91,7 @@ public final class MultiJoin extends AbstractRelNode {
       boolean isFullOuterJoin,
       List<RexNode> outerJoinConditions,
       List<JoinRelType> joinTypes,
-      List<BitSet> projFields,
+      List<ImmutableBitSet> projFields,
       ImmutableMap<Integer, ImmutableIntList> joinFieldRefCountsMap,
       RexNode postJoinFilter) {
     super(cluster, cluster.traitSetOf(Convention.NONE));
@@ -214,7 +214,7 @@ public final class MultiJoin extends AbstractRelNode {
    * @return bitmaps representing the fields projected from each input; if an
    * entry is null, all fields are projected
    */
-  public List<BitSet> getProjFields() {
+  public List<ImmutableBitSet> getProjFields() {
     return projFields;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/core/src/main/java/org/apache/calcite/rel/rules/MultiJoinOptimizeBushyRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/MultiJoinOptimizeBushyRule.java b/core/src/main/java/org/apache/calcite/rel/rules/MultiJoinOptimizeBushyRule.java
index 62eef89..2e6667c 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/MultiJoinOptimizeBushyRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/MultiJoinOptimizeBushyRule.java
@@ -30,7 +30,7 @@ import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexPermuteInputsShuttle;
 import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.rex.RexVisitor;
-import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Util;
 import org.apache.calcite.util.mapping.Mappings;
@@ -41,7 +41,6 @@ import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Lists;
 
 import java.io.PrintWriter;
-import java.util.BitSet;
 import java.util.Comparator;
 import java.util.Iterator;
 import java.util.List;
@@ -132,8 +131,7 @@ public class MultiJoinOptimizeBushyRule extends RelOptRule {
       if (edgeOrdinal == -1) {
         // No more edges. Are there any un-joined vertexes?
         final Vertex lastVertex = Util.last(vertexes);
-        final int z =
-            BitSets.previousClearBit(lastVertex.factors, lastVertex.id - 1);
+        final int z = lastVertex.factors.previousClearBit(lastVertex.id - 1);
         if (z < 0) {
           break;
         }
@@ -147,7 +145,7 @@ public class MultiJoinOptimizeBushyRule extends RelOptRule {
         // Therefore, for now, the factors that are merged are exactly the
         // factors on this edge.
         assert bestEdge.factors.cardinality() == 2;
-        factors = BitSets.toArray(bestEdge.factors);
+        factors = bestEdge.factors.toArray();
       }
 
       // Determine which factor is to be on the LHS of the join.
@@ -165,26 +163,29 @@ public class MultiJoinOptimizeBushyRule extends RelOptRule {
 
       // Find the join conditions. All conditions whose factors are now all in
       // the join can now be used.
-      final BitSet newFactors =
-          BitSets.union(majorVertex.factors, minorVertex.factors);
+      final int v = vertexes.size();
+      final ImmutableBitSet newFactors =
+          ImmutableBitSet.builder(majorVertex.factors)
+              .addAll(minorVertex.factors)
+              .set(v)
+              .build();
+
       final List<RexNode> conditions = Lists.newArrayList();
       final Iterator<LoptMultiJoin.Edge> edgeIterator = unusedEdges.iterator();
       while (edgeIterator.hasNext()) {
         LoptMultiJoin.Edge edge = edgeIterator.next();
-        if (BitSets.contains(newFactors, edge.factors)) {
+        if (newFactors.contains(edge.factors)) {
           conditions.add(edge.condition);
           edgeIterator.remove();
           usedEdges.add(edge);
         }
       }
 
-      final int v = vertexes.size();
       double cost =
           majorVertex.cost
           * minorVertex.cost
           * RelMdUtil.guessSelectivity(
               RexUtil.composeConjunction(rexBuilder, conditions, false));
-      newFactors.set(v);
       final Vertex newVertex =
           new JoinVertex(v, majorFactor, minorFactor, newFactors,
               cost, ImmutableList.copyOf(conditions));
@@ -197,13 +198,16 @@ public class MultiJoinOptimizeBushyRule extends RelOptRule {
       // This vertex has fewer rows (1k rows) -- a fact that is critical to
       // decisions made later. (Hence "greedy" algorithm not "simple".)
       // The adjacent edges are modified.
-      final BitSet merged = BitSets.of(minorFactor, majorFactor);
+      final ImmutableBitSet merged =
+          ImmutableBitSet.of(minorFactor, majorFactor);
       for (int i = 0; i < unusedEdges.size(); i++) {
         final LoptMultiJoin.Edge edge = unusedEdges.get(i);
         if (edge.factors.intersects(merged)) {
-          BitSet newEdgeFactors = (BitSet) edge.factors.clone();
-          newEdgeFactors.andNot(newFactors);
-          newEdgeFactors.set(v);
+          ImmutableBitSet newEdgeFactors =
+              ImmutableBitSet.builder(edge.factors)
+              .removeAll(newFactors)
+              .set(v)
+              .build();
           assert newEdgeFactors.cardinality() == 2;
           final LoptMultiJoin.Edge newEdge =
               new LoptMultiJoin.Edge(edge.condition, newEdgeFactors,
@@ -251,7 +255,7 @@ public class MultiJoinOptimizeBushyRule extends RelOptRule {
             RexUtil.composeConjunction(rexBuilder, joinVertex.conditions,
                 false);
         relNodes.add(
-            Pair.of((RelNode)
+            Pair.of(
                 joinFactory.createJoin(left, right, condition.accept(shuttle),
                     JoinRelType.INNER, ImmutableSet.<String>of(), false),
                 mapping));
@@ -321,10 +325,10 @@ public class MultiJoinOptimizeBushyRule extends RelOptRule {
   abstract static class Vertex {
     final int id;
 
-    protected final BitSet factors;
+    protected final ImmutableBitSet factors;
     final double cost;
 
-    Vertex(int id, BitSet factors, double cost) {
+    Vertex(int id, ImmutableBitSet factors, double cost) {
       this.id = id;
       this.factors = factors;
       this.cost = cost;
@@ -337,7 +341,7 @@ public class MultiJoinOptimizeBushyRule extends RelOptRule {
     final int fieldOffset;
 
     LeafVertex(int id, RelNode rel, double cost, int fieldOffset) {
-      super(id, BitSets.of(id), cost);
+      super(id, ImmutableBitSet.of(id), cost);
       this.rel = rel;
       this.fieldOffset = fieldOffset;
     }
@@ -359,8 +363,8 @@ public class MultiJoinOptimizeBushyRule extends RelOptRule {
      * columns (not in terms of the outputs of left and right input factors). */
     final ImmutableList<RexNode> conditions;
 
-    JoinVertex(int id, int leftFactor, int rightFactor,
-        BitSet factors, double cost, ImmutableList<RexNode> conditions) {
+    JoinVertex(int id, int leftFactor, int rightFactor, ImmutableBitSet factors,
+        double cost, ImmutableList<RexNode> conditions) {
       super(id, factors, cost);
       this.leftFactor = leftFactor;
       this.rightFactor = rightFactor;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/core/src/main/java/org/apache/calcite/rel/rules/PushProjector.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/PushProjector.java b/core/src/main/java/org/apache/calcite/rel/rules/PushProjector.java
index d317237..cfaf740 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/PushProjector.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/PushProjector.java
@@ -31,6 +31,7 @@ import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.rex.RexVisitorImpl;
 import org.apache.calcite.sql.SqlOperator;
 import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.Pair;
 
 import com.google.common.collect.ImmutableList;
@@ -88,14 +89,14 @@ public class PushProjector {
    * pushed past, if the RelNode is not a join. If the RelNode is a join, then
    * the fields correspond to the left hand side of the join.
    */
-  final BitSet childBitmap;
+  final ImmutableBitSet childBitmap;
 
   /**
    * Bitmap containing the fields in the right hand side of a join, in the
    * case where the projection is being pushed past a join. Not used
    * otherwise.
    */
-  final BitSet rightBitmap;
+  final ImmutableBitSet rightBitmap;
 
   /**
    * Number of fields in the RelNode that the projection is being pushed past,
@@ -215,13 +216,13 @@ public class PushProjector {
       nFieldsRight = rightFields.size();
       nSysFields = joinRel.getSystemFieldList().size();
       childBitmap =
-          BitSets.range(nSysFields, nFields + nSysFields);
+          ImmutableBitSet.range(nSysFields, nFields + nSysFields);
       rightBitmap =
-          BitSets.range(nFields + nSysFields, nChildFields);
+          ImmutableBitSet.range(nFields + nSysFields, nChildFields);
     } else {
       nFields = nChildFields;
       nFieldsRight = 0;
-      childBitmap = BitSets.range(nChildFields);
+      childBitmap = ImmutableBitSet.range(nChildFields);
       rightBitmap = null;
       nSysFields = 0;
     }
@@ -574,16 +575,16 @@ public class PushProjector {
    */
   private class InputSpecialOpFinder extends RexVisitorImpl<Void> {
     private final BitSet rexRefs;
-    private final BitSet leftFields;
-    private final BitSet rightFields;
+    private final ImmutableBitSet leftFields;
+    private final ImmutableBitSet rightFields;
     private final ExprCondition preserveExprCondition;
     private final List<RexNode> preserveLeft;
     private final List<RexNode> preserveRight;
 
     public InputSpecialOpFinder(
         BitSet rexRefs,
-        BitSet leftFields,
-        BitSet rightFields,
+        ImmutableBitSet leftFields,
+        ImmutableBitSet rightFields,
         ExprCondition preserveExprCondition,
         List<RexNode> preserveLeft,
         List<RexNode> preserveRight) {
@@ -609,12 +610,12 @@ public class PushProjector {
         // if the arguments of the expression only reference the
         // left hand side, preserve it on the left; similarly, if
         // it only references expressions on the right
-        final BitSet exprArgs = RelOptUtil.InputFinder.bits(call);
+        final ImmutableBitSet exprArgs = RelOptUtil.InputFinder.bits(call);
         if (exprArgs.cardinality() > 0) {
-          if (BitSets.contains(leftFields, exprArgs)) {
+          if (leftFields.contains(exprArgs)) {
             addExpr(preserveLeft, call);
             return true;
-          } else if (BitSets.contains(rightFields, exprArgs)) {
+          } else if (rightFields.contains(exprArgs)) {
             assert preserveRight != null;
             addExpr(preserveRight, call);
             return true;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/core/src/main/java/org/apache/calcite/rel/rules/SemiJoinRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/SemiJoinRule.java b/core/src/main/java/org/apache/calcite/rel/rules/SemiJoinRule.java
index bedea88..28d9454 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/SemiJoinRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/SemiJoinRule.java
@@ -26,13 +26,12 @@ import org.apache.calcite.rel.core.Join;
 import org.apache.calcite.rel.core.JoinInfo;
 import org.apache.calcite.rel.core.Project;
 import org.apache.calcite.rel.core.SemiJoin;
-import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.ImmutableIntList;
 import org.apache.calcite.util.IntList;
 
 import com.google.common.collect.Lists;
 
-import java.util.BitSet;
 import java.util.List;
 
 /**
@@ -56,21 +55,23 @@ public class SemiJoinRule extends RelOptRule {
     final Join join = call.rel(1);
     final RelNode left = call.rel(2);
     final Aggregate aggregate = call.rel(3);
-    final BitSet bits = RelOptUtil.InputFinder.bits(project.getProjects(),
-        null);
-    final BitSet rightBits = BitSets.range(left.getRowType().getFieldCount(),
-        join.getRowType().getFieldCount());
+    final ImmutableBitSet bits =
+        RelOptUtil.InputFinder.bits(project.getProjects(), null);
+    final ImmutableBitSet rightBits =
+        ImmutableBitSet.range(left.getRowType().getFieldCount(),
+            join.getRowType().getFieldCount());
     if (bits.intersects(rightBits)) {
       return;
     }
     final JoinInfo joinInfo = join.analyzeCondition();
-    if (!joinInfo.rightSet().equals(BitSets.range(aggregate.getGroupCount()))) {
+    if (!joinInfo.rightSet().equals(
+        ImmutableBitSet.range(aggregate.getGroupCount()))) {
       // Rule requires that aggregate key to be the same as the join key.
       // By the way, neither a super-set nor a sub-set would work.
       return;
     }
     final List<Integer> newRightKeys = Lists.newArrayList();
-    final IntList aggregateKeys = BitSets.toList(aggregate.getGroupSet());
+    final IntList aggregateKeys = aggregate.getGroupSet().toList();
     for (int key : joinInfo.rightKeys) {
       newRightKeys.add(aggregateKeys.get(key));
     }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/core/src/main/java/org/apache/calcite/schema/Statistic.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/schema/Statistic.java b/core/src/main/java/org/apache/calcite/schema/Statistic.java
index fa907da..b725d4b 100644
--- a/core/src/main/java/org/apache/calcite/schema/Statistic.java
+++ b/core/src/main/java/org/apache/calcite/schema/Statistic.java
@@ -16,7 +16,7 @@
  */
 package org.apache.calcite.schema;
 
-import java.util.BitSet;
+import org.apache.calcite.util.ImmutableBitSet;
 
 /**
  * Statistics about a {@link Table}.
@@ -32,7 +32,7 @@ public interface Statistic {
   /** Returns whether the given set of columns is a unique key, or a superset
    * of a unique key, of the table.
    */
-  boolean isKey(BitSet columns);
+  boolean isKey(ImmutableBitSet columns);
 }
 
 // End Statistic.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/core/src/main/java/org/apache/calcite/schema/Statistics.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/schema/Statistics.java b/core/src/main/java/org/apache/calcite/schema/Statistics.java
index 9712e7c..4b2611a 100644
--- a/core/src/main/java/org/apache/calcite/schema/Statistics.java
+++ b/core/src/main/java/org/apache/calcite/schema/Statistics.java
@@ -16,9 +16,8 @@
  */
 package org.apache.calcite.schema;
 
-import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.ImmutableBitSet;
 
-import java.util.BitSet;
 import java.util.List;
 
 /**
@@ -35,21 +34,22 @@ public class Statistics {
           return null;
         }
 
-        public boolean isKey(BitSet columns) {
+        public boolean isKey(ImmutableBitSet columns) {
           return false;
         }
       };
 
   /** Returns a statistic with a given row count and set of unique keys. */
-  public static Statistic of(final double rowCount, final List<BitSet> keys) {
+  public static Statistic of(final double rowCount,
+      final List<ImmutableBitSet> keys) {
     return new Statistic() {
       public Double getRowCount() {
         return rowCount;
       }
 
-      public boolean isKey(BitSet columns) {
-        for (BitSet key : keys) {
-          if (BitSets.contains(columns, key)) {
+      public boolean isKey(ImmutableBitSet columns) {
+        for (ImmutableBitSet key : keys) {
+          if (columns.contains(key)) {
             return true;
           }
         }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/core/src/main/java/org/apache/calcite/sql2rel/RelDecorrelator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql2rel/RelDecorrelator.java b/core/src/main/java/org/apache/calcite/sql2rel/RelDecorrelator.java
index 4da6445..6da2268 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/RelDecorrelator.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/RelDecorrelator.java
@@ -64,9 +64,9 @@ import org.apache.calcite.sql.SqlOperator;
 import org.apache.calcite.sql.fun.SqlCountAggFunction;
 import org.apache.calcite.sql.fun.SqlSingleValueAggFunction;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
-import org.apache.calcite.util.BitSets;
 import org.apache.calcite.util.Bug;
 import org.apache.calcite.util.Holder;
+import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.ReflectUtil;
 import org.apache.calcite.util.ReflectiveVisitDispatcher;
@@ -87,7 +87,6 @@ import com.google.common.collect.SortedSetMultimap;
 
 import java.math.BigDecimal;
 import java.util.ArrayList;
-import java.util.BitSet;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
@@ -582,7 +581,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
         new LogicalAggregate(
             rel.getCluster(),
             newProjectRel,
-            BitSets.range(newGroupKeyCount),
+            ImmutableBitSet.range(newGroupKeyCount),
             newAggCalls);
 
     mapOldToNewRel.put(rel, newAggregate);
@@ -2202,7 +2201,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
         }
 
         int nFields = leftInputRel.getRowType().getFieldCount();
-        BitSet allCols = BitSets.range(nFields);
+        ImmutableBitSet allCols = ImmutableBitSet.range(nFields);
 
         // leftInputRel contains unique keys
         // i.e. each row is distinct and can group by on all the left
@@ -2345,8 +2344,8 @@ public class RelDecorrelator implements ReflectiveVisitor {
                 aggRel.getGroupCount(), groupCount));
       }
 
-      BitSet groupSet =
-          BitSets.range(groupCount);
+      ImmutableBitSet groupSet =
+          ImmutableBitSet.range(groupCount);
       LogicalAggregate newAggRel =
           new LogicalAggregate(
               cluster,
@@ -2355,7 +2354,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
               newAggCalls);
 
       List<RexNode> newAggOutputProjExprList = Lists.newArrayList();
-      for (int i : BitSets.toIter(groupSet)) {
+      for (int i : groupSet) {
         newAggOutputProjExprList.add(
             rexBuilder.makeInputRef(newAggRel, i));
       }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/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 075def1..c68273d 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/RelFieldTrimmer.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/RelFieldTrimmer.java
@@ -47,8 +47,8 @@ import org.apache.calcite.rex.RexPermuteInputsShuttle;
 import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.rex.RexVisitor;
 import org.apache.calcite.sql.validate.SqlValidator;
-import org.apache.calcite.util.BitSets;
 import org.apache.calcite.util.Bug;
+import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.ReflectUtil;
 import org.apache.calcite.util.ReflectiveVisitor;
@@ -63,7 +63,6 @@ import com.google.common.collect.ImmutableList;
 
 import java.math.BigDecimal;
 import java.util.ArrayList;
-import java.util.BitSet;
 import java.util.Collections;
 import java.util.LinkedHashSet;
 import java.util.List;
@@ -144,7 +143,7 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
             this,
             "trimFields",
             RelNode.class,
-            BitSet.class,
+            ImmutableBitSet.class,
             Set.class);
     this.projectFactory = Preconditions.checkNotNull(projectFactory);
     this.filterFactory = Preconditions.checkNotNull(filterFactory);
@@ -168,7 +167,7 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
    */
   public RelNode trim(RelNode root) {
     final int fieldCount = root.getRowType().getFieldCount();
-    final BitSet fieldsUsed = BitSets.range(fieldCount);
+    final ImmutableBitSet fieldsUsed = ImmutableBitSet.range(fieldCount);
     final Set<RelDataTypeField> extraFields = Collections.emptySet();
     final TrimResult trimResult =
         dispatchTrimFields(root, fieldsUsed, extraFields);
@@ -189,14 +188,14 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
   protected TrimResult trimChild(
       RelNode rel,
       RelNode input,
-      BitSet fieldsUsed,
+      ImmutableBitSet fieldsUsed,
       Set<RelDataTypeField> extraFields) {
     Util.discard(rel);
     if (input.getClass().getName().endsWith("MedMdrClassExtentRel")) {
       // MedMdrJoinRule cannot handle Join of Project of
       // MedMdrClassExtentRel, only naked MedMdrClassExtentRel.
       // So, disable trimming.
-      fieldsUsed = BitSets.range(input.getRowType().getFieldCount());
+      fieldsUsed = ImmutableBitSet.range(input.getRowType().getFieldCount());
     }
     return dispatchTrimFields(input, fieldsUsed, extraFields);
   }
@@ -217,7 +216,7 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
   protected TrimResult trimChildRestore(
       RelNode rel,
       RelNode input,
-      BitSet fieldsUsed,
+      ImmutableBitSet fieldsUsed,
       Set<RelDataTypeField> extraFields) {
     TrimResult trimResult = trimChild(rel, input, fieldsUsed, extraFields);
     if (trimResult.right.isIdentity()) {
@@ -255,7 +254,7 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
    */
   protected final TrimResult dispatchTrimFields(
       RelNode rel,
-      BitSet fieldsUsed,
+      ImmutableBitSet fieldsUsed,
       Set<RelDataTypeField> extraFields) {
     final TrimResult trimResult =
         trimFieldsDispatcher.invoke(rel, fieldsUsed, extraFields);
@@ -296,7 +295,7 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
    */
   public TrimResult trimFields(
       RelNode rel,
-      BitSet fieldsUsed,
+      ImmutableBitSet fieldsUsed,
       Set<RelDataTypeField> extraFields) {
     // We don't know how to trim this kind of relational expression, so give
     // it back intact.
@@ -308,12 +307,12 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
   }
 
   /**
-   * Variant of {@link #trimFields(RelNode, BitSet, Set)} for
+   * Variant of {@link #trimFields(RelNode, ImmutableBitSet, Set)} for
    * {@link org.apache.calcite.rel.logical.LogicalProject}.
    */
   public TrimResult trimFields(
       Project project,
-      BitSet fieldsUsed,
+      ImmutableBitSet fieldsUsed,
       Set<RelDataTypeField> extraFields) {
     final RelDataType rowType = project.getRowType();
     final int fieldCount = rowType.getFieldCount();
@@ -321,16 +320,16 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
     final RelDataType inputRowType = input.getRowType();
 
     // Which fields are required from the input?
-    BitSet inputFieldsUsed = new BitSet(inputRowType.getFieldCount());
     final Set<RelDataTypeField> inputExtraFields =
         new LinkedHashSet<RelDataTypeField>(extraFields);
     RelOptUtil.InputFinder inputFinder =
-        new RelOptUtil.InputFinder(inputFieldsUsed, inputExtraFields);
+        new RelOptUtil.InputFinder(inputExtraFields);
     for (Ord<RexNode> ord : Ord.zip(project.getProjects())) {
       if (fieldsUsed.get(ord.i)) {
         ord.e.accept(inputFinder);
       }
     }
+    ImmutableBitSet inputFieldsUsed = inputFinder.inputBitSet.build();
 
     // Create input with trimmed columns.
     TrimResult trimResult =
@@ -416,12 +415,12 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
   }
 
   /**
-   * Variant of {@link #trimFields(RelNode, BitSet, Set)} for
+   * Variant of {@link #trimFields(RelNode, ImmutableBitSet, Set)} for
    * {@link org.apache.calcite.rel.logical.LogicalFilter}.
    */
   public TrimResult trimFields(
       Filter filter,
-      BitSet fieldsUsed,
+      ImmutableBitSet fieldsUsed,
       Set<RelDataTypeField> extraFields) {
     final RelDataType rowType = filter.getRowType();
     final int fieldCount = rowType.getFieldCount();
@@ -430,12 +429,13 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
 
     // We use the fields used by the consumer, plus any fields used in the
     // filter.
-    BitSet inputFieldsUsed = (BitSet) fieldsUsed.clone();
     final Set<RelDataTypeField> inputExtraFields =
         new LinkedHashSet<RelDataTypeField>(extraFields);
     RelOptUtil.InputFinder inputFinder =
-        new RelOptUtil.InputFinder(inputFieldsUsed, inputExtraFields);
+        new RelOptUtil.InputFinder(inputExtraFields);
+    inputFinder.inputBitSet.addAll(fieldsUsed);
     conditionExpr.accept(inputFinder);
+    final ImmutableBitSet inputFieldsUsed = inputFinder.inputBitSet.build();
 
     // Create input with trimmed columns.
     TrimResult trimResult =
@@ -468,12 +468,12 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
   }
 
   /**
-   * Variant of {@link #trimFields(RelNode, BitSet, Set)} for
+   * Variant of {@link #trimFields(RelNode, ImmutableBitSet, Set)} for
    * {@link org.apache.calcite.rel.core.Sort}.
    */
   public TrimResult trimFields(
       Sort sort,
-      BitSet fieldsUsed,
+      ImmutableBitSet fieldsUsed,
       Set<RelDataTypeField> extraFields) {
     final RelDataType rowType = sort.getRowType();
     final int fieldCount = rowType.getFieldCount();
@@ -482,7 +482,8 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
 
     // We use the fields used by the consumer, plus any fields used as sort
     // keys.
-    BitSet inputFieldsUsed = (BitSet) fieldsUsed.clone();
+    final ImmutableBitSet.Builder inputFieldsUsed =
+        ImmutableBitSet.builder(fieldsUsed);
     for (RelFieldCollation field : collation.getFieldCollations()) {
       inputFieldsUsed.set(field.getFieldIndex());
     }
@@ -490,7 +491,7 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
     // Create input with trimmed columns.
     final Set<RelDataTypeField> inputExtraFields = Collections.emptySet();
     TrimResult trimResult =
-        trimChild(sort, input, inputFieldsUsed, inputExtraFields);
+        trimChild(sort, input, inputFieldsUsed.build(), inputExtraFields);
     RelNode newInput = trimResult.left;
     final Mapping inputMapping = trimResult.right;
 
@@ -517,12 +518,12 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
   }
 
   /**
-   * Variant of {@link #trimFields(RelNode, BitSet, Set)} for
+   * Variant of {@link #trimFields(RelNode, ImmutableBitSet, Set)} for
    * {@link org.apache.calcite.rel.logical.LogicalJoin}.
    */
   public TrimResult trimFields(
       Join join,
-      BitSet fieldsUsed,
+      ImmutableBitSet fieldsUsed,
       Set<RelDataTypeField> extraFields) {
     final int fieldCount = join.getSystemFieldList().size()
         + join.getLeft().getRowType().getFieldCount()
@@ -531,13 +532,13 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
     final int systemFieldCount = join.getSystemFieldList().size();
 
     // Add in fields used in the condition.
-    BitSet fieldsUsedPlus = (BitSet) fieldsUsed.clone();
     final Set<RelDataTypeField> combinedInputExtraFields =
         new LinkedHashSet<RelDataTypeField>(extraFields);
     RelOptUtil.InputFinder inputFinder =
-        new RelOptUtil.InputFinder(
-            fieldsUsedPlus, combinedInputExtraFields);
+        new RelOptUtil.InputFinder(combinedInputExtraFields);
+    inputFinder.inputBitSet.addAll(fieldsUsed);
     conditionExpr.accept(inputFinder);
+    final ImmutableBitSet fieldsUsedPlus = inputFinder.inputBitSet.build();
 
     // If no system fields are used, we can remove them.
     int systemFieldUsedCount = 0;
@@ -564,8 +565,8 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
       final int inputFieldCount = inputRowType.getFieldCount();
 
       // Compute required mapping.
-      BitSet inputFieldsUsed = new BitSet(inputFieldCount);
-      for (int bit : BitSets.toIter(fieldsUsedPlus)) {
+      ImmutableBitSet.Builder inputFieldsUsed = ImmutableBitSet.builder();
+      for (int bit : fieldsUsedPlus) {
         if (bit >= offset && bit < offset + inputFieldCount) {
           inputFieldsUsed.set(bit - offset);
         }
@@ -573,11 +574,7 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
 
       // If there are system fields, we automatically use the
       // corresponding field in each input.
-      if (newSystemFieldCount > 0) {
-        // calling with newSystemFieldCount == 0 should be safe but hits
-        // http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=6222207
-        inputFieldsUsed.set(0, newSystemFieldCount);
-      }
+      inputFieldsUsed.set(0, newSystemFieldCount);
 
       // FIXME: We ought to collect extra fields for each input
       // individually. For now, we assume that just one input has
@@ -588,7 +585,7 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
               : combinedInputExtraFields;
       inputExtraFieldCounts.add(inputExtraFields.size());
       TrimResult trimResult =
-          trimChild(join, input, inputFieldsUsed, inputExtraFields);
+          trimChild(join, input, inputFieldsUsed.build(), inputExtraFields);
       newInputs.add(trimResult.left);
       if (trimResult.left != input) {
         ++changeCount;
@@ -662,12 +659,12 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
   }
 
   /**
-   * Variant of {@link #trimFields(RelNode, BitSet, Set)} for
+   * Variant of {@link #trimFields(RelNode, ImmutableBitSet, Set)} for
    * {@link org.apache.calcite.rel.core.SetOp} (including UNION and UNION ALL).
    */
   public TrimResult trimFields(
       SetOp setOp,
-      BitSet fieldsUsed,
+      ImmutableBitSet fieldsUsed,
       Set<RelDataTypeField> extraFields) {
     final RelDataType rowType = setOp.getRowType();
     final int fieldCount = rowType.getFieldCount();
@@ -677,7 +674,7 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
     // wants the last field. (The last field is the least likely to be a
     // system field.)
     if (fieldsUsed.isEmpty()) {
-      fieldsUsed.set(rowType.getFieldCount() - 1);
+      fieldsUsed = ImmutableBitSet.of(rowType.getFieldCount() - 1);
     }
 
     // Compute the desired field mapping. Give the consumer the fields they
@@ -730,12 +727,12 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
   }
 
   /**
-   * Variant of {@link #trimFields(RelNode, BitSet, Set)} for
+   * Variant of {@link #trimFields(RelNode, ImmutableBitSet, Set)} for
    * {@link org.apache.calcite.rel.logical.LogicalAggregate}.
    */
   public TrimResult trimFields(
       Aggregate aggregate,
-      BitSet fieldsUsed,
+      ImmutableBitSet fieldsUsed,
       Set<RelDataTypeField> extraFields) {
     // Fields:
     //
@@ -753,11 +750,9 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
     final RelDataType rowType = aggregate.getRowType();
 
     // Compute which input fields are used.
-    BitSet inputFieldsUsed = new BitSet();
     // 1. group fields are always used
-    for (int i : BitSets.toIter(aggregate.getGroupSet())) {
-      inputFieldsUsed.set(i);
-    }
+    final ImmutableBitSet.Builder inputFieldsUsed =
+        ImmutableBitSet.builder(aggregate.getGroupSet());
     // 2. agg functions
     for (AggregateCall aggCall : aggregate.getAggCallList()) {
       for (int i : aggCall.getArgList()) {
@@ -769,14 +764,14 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
     final RelNode input = aggregate.getInput(0);
     final Set<RelDataTypeField> inputExtraFields = Collections.emptySet();
     final TrimResult trimResult =
-        trimChild(aggregate, input, inputFieldsUsed, inputExtraFields);
+        trimChild(aggregate, input, inputFieldsUsed.build(), inputExtraFields);
     final RelNode newInput = trimResult.left;
     final Mapping inputMapping = trimResult.right;
 
     // If the input is unchanged, and we need to project all columns,
     // there's nothing to do.
     if (input == newInput
-        && fieldsUsed.equals(BitSets.range(rowType.getFieldCount()))) {
+        && fieldsUsed.equals(ImmutableBitSet.range(rowType.getFieldCount()))) {
       return new TrimResult(
           aggregate,
           Mappings.createIdentity(rowType.getFieldCount()));
@@ -800,7 +795,7 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
             groupCount
                 + usedAggCallCount);
 
-    final BitSet newGroupSet =
+    final ImmutableBitSet newGroupSet =
         Mappings.apply(inputMapping, aggregate.getGroupSet());
 
     // Populate mapping of where to find the fields. System and grouping
@@ -840,12 +835,12 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
   }
 
   /**
-   * Variant of {@link #trimFields(RelNode, BitSet, Set)} for
+   * Variant of {@link #trimFields(RelNode, ImmutableBitSet, Set)} for
    * {@link org.apache.calcite.rel.logical.LogicalTableModify}.
    */
   public TrimResult trimFields(
       LogicalTableModify modifier,
-      BitSet fieldsUsed,
+      ImmutableBitSet fieldsUsed,
       Set<RelDataTypeField> extraFields) {
     // Ignore what consumer wants. We always project all columns.
     Util.discard(fieldsUsed);
@@ -856,7 +851,8 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
 
     // We want all fields from the child.
     final int inputFieldCount = input.getRowType().getFieldCount();
-    BitSet inputFieldsUsed = BitSets.range(inputFieldCount);
+    final ImmutableBitSet inputFieldsUsed =
+        ImmutableBitSet.range(inputFieldCount);
 
     // Create input with trimmed columns.
     final Set<RelDataTypeField> inputExtraFields = Collections.emptySet();
@@ -886,12 +882,12 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
   }
 
   /**
-   * Variant of {@link #trimFields(RelNode, BitSet, Set)} for
+   * Variant of {@link #trimFields(RelNode, ImmutableBitSet, Set)} for
    * {@link org.apache.calcite.rel.logical.LogicalTableFunctionScan}.
    */
   public TrimResult trimFields(
       LogicalTableFunctionScan tabFun,
-      BitSet fieldsUsed,
+      ImmutableBitSet fieldsUsed,
       Set<RelDataTypeField> extraFields) {
     final RelDataType rowType = tabFun.getRowType();
     final int fieldCount = rowType.getFieldCount();
@@ -899,7 +895,7 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
 
     for (RelNode input : tabFun.getInputs()) {
       final int inputFieldCount = input.getRowType().getFieldCount();
-      BitSet inputFieldsUsed = BitSets.range(inputFieldCount);
+      ImmutableBitSet inputFieldsUsed = ImmutableBitSet.range(inputFieldCount);
 
       // Create input with trimmed columns.
       final Set<RelDataTypeField> inputExtraFields =
@@ -923,12 +919,12 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
   }
 
   /**
-   * Variant of {@link #trimFields(RelNode, BitSet, Set)} for
+   * Variant of {@link #trimFields(RelNode, ImmutableBitSet, Set)} for
    * {@link org.apache.calcite.rel.logical.LogicalValues}.
    */
   public TrimResult trimFields(
       LogicalValues values,
-      BitSet fieldsUsed,
+      ImmutableBitSet fieldsUsed,
       Set<RelDataTypeField> extraFields) {
     final RelDataType rowType = values.getRowType();
     final int fieldCount = rowType.getFieldCount();
@@ -937,11 +933,11 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
     // because zero-column records are illegal. Give them the last field,
     // which is unlikely to be a system field.
     if (fieldsUsed.isEmpty()) {
-      fieldsUsed = BitSets.range(fieldCount - 1, fieldCount);
+      fieldsUsed = ImmutableBitSet.range(fieldCount - 1, fieldCount);
     }
 
     // If all fields are used, return unchanged.
-    if (fieldsUsed.equals(BitSets.range(fieldCount))) {
+    if (fieldsUsed.equals(ImmutableBitSet.range(fieldCount))) {
       Mapping mapping = Mappings.createIdentity(fieldCount);
       return new TrimResult(values, mapping);
     }
@@ -949,7 +945,7 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
     List<List<RexLiteral>> newTuples = new ArrayList<List<RexLiteral>>();
     for (List<RexLiteral> tuple : values.getTuples()) {
       List<RexLiteral> newTuple = new ArrayList<RexLiteral>();
-      for (int field : BitSets.toIter(fieldsUsed)) {
+      for (int field : fieldsUsed) {
         newTuple.add(tuple.get(field));
       }
       newTuples.add(newTuple);
@@ -964,29 +960,30 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
     return new TrimResult(newValues, mapping);
   }
 
-  private Mapping createMapping(BitSet fieldsUsed, int fieldCount) {
+  private Mapping createMapping(ImmutableBitSet fieldsUsed, int fieldCount) {
     final Mapping mapping =
         Mappings.create(
             MappingType.INVERSE_SURJECTION,
             fieldCount,
             fieldsUsed.cardinality());
     int i = 0;
-    for (int field : BitSets.toIter(fieldsUsed)) {
+    for (int field : fieldsUsed) {
       mapping.set(field, i++);
     }
     return mapping;
   }
 
   /**
-   * Variant of {@link #trimFields(RelNode, BitSet, Set)} for
+   * Variant of {@link #trimFields(RelNode, ImmutableBitSet, Set)} for
    * {@link org.apache.calcite.rel.logical.LogicalTableScan}.
    */
   public TrimResult trimFields(
       final TableScan tableAccessRel,
-      BitSet fieldsUsed,
+      ImmutableBitSet fieldsUsed,
       Set<RelDataTypeField> extraFields) {
     final int fieldCount = tableAccessRel.getRowType().getFieldCount();
-    if (fieldsUsed.equals(BitSets.range(fieldCount)) && extraFields.isEmpty()) {
+    if (fieldsUsed.equals(ImmutableBitSet.range(fieldCount))
+        && extraFields.isEmpty()) {
       // if there is nothing to project or if we are projecting everything
       // then no need to introduce another RelNode
       return trimFields(

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java b/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
index ce1cb4a..ceb6a1a 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
@@ -127,7 +127,7 @@ import org.apache.calcite.sql.validate.SqlValidatorImpl;
 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.BitSets;
+import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.ImmutableIntList;
 import org.apache.calcite.util.NlsString;
 import org.apache.calcite.util.NumberUtil;
@@ -148,7 +148,6 @@ import java.lang.reflect.Type;
 import java.math.BigDecimal;
 import java.util.AbstractList;
 import java.util.ArrayList;
-import java.util.BitSet;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.EnumSet;
@@ -714,7 +713,7 @@ public class SqlToRelConverter {
     rel =
         createAggregate(
             bb,
-            BitSets.range(rel.getRowType().getFieldCount()),
+            ImmutableBitSet.range(rel.getRowType().getFieldCount()),
             ImmutableList.<AggregateCall>of());
 
     bb.setRoot(
@@ -1033,7 +1032,7 @@ public class SqlToRelConverter {
         final int keyCount = leftKeys.size();
         final List<Integer> args = ImmutableIntList.range(0, keyCount);
         LogicalAggregate aggregate =
-            new LogicalAggregate(cluster, seek, BitSets.of(),
+            new LogicalAggregate(cluster, seek, ImmutableBitSet.of(),
                 ImmutableList.of(
                     new AggregateCall(SqlStdOperatorTable.COUNT, false,
                         ImmutableList.<Integer>of(), longType, null),
@@ -2276,7 +2275,7 @@ public class SqlToRelConverter {
     case LITERAL:
       return node;
     default:
-      BitSet bits = RelOptUtil.InputFinder.bits(node);
+      ImmutableBitSet bits = RelOptUtil.InputFinder.bits(node);
       final int mid = leftCount + extraLeftExprs.size();
       switch (Side.of(bits, mid)) {
       case LEFT:
@@ -2312,7 +2311,7 @@ public class SqlToRelConverter {
   enum Side {
     LEFT, RIGHT, BOTH, EMPTY;
 
-    static Side of(BitSet bitSet, int middle) {
+    static Side of(ImmutableBitSet bitSet, int middle) {
       final int firstBit = bitSet.nextSetBit(0);
       if (firstBit < 0) {
         return EMPTY;
@@ -2646,7 +2645,7 @@ public class SqlToRelConverter {
       bb.setRoot(
           createAggregate(
               bb,
-              BitSets.range(aggConverter.groupExprs.size()),
+              ImmutableBitSet.range(aggConverter.groupExprs.size()),
               aggConverter.getAggCalls()),
           false);
 
@@ -2744,7 +2743,7 @@ public class SqlToRelConverter {
    */
   protected RelNode createAggregate(
       Blackboard bb,
-      BitSet groupSet,
+      ImmutableBitSet groupSet,
       List<AggregateCall> aggCalls) {
     return new LogicalAggregate(
         cluster,

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/core/src/main/java/org/apache/calcite/util/BitSets.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/BitSets.java b/core/src/main/java/org/apache/calcite/util/BitSets.java
index 0d79c85..bbab792 100644
--- a/core/src/main/java/org/apache/calcite/util/BitSets.java
+++ b/core/src/main/java/org/apache/calcite/util/BitSets.java
@@ -48,6 +48,24 @@ public final class BitSets {
   }
 
   /**
+   * Returns true if all bits set in the second parameter are also set in the
+   * first. In other words, whether x is a super-set of y.
+   *
+   * @param set0 Containing bitmap
+   * @param set1 Bitmap to be checked
+   *
+   * @return Whether all bits in set1 are set in set0
+   */
+  public static boolean contains(BitSet set0, ImmutableBitSet set1) {
+    for (int i = set1.nextSetBit(0); i >= 0; i = set1.nextSetBit(i + 1)) {
+      if (!set0.get(i)) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  /**
    * Returns an iterable over the bits in a bitmap that are set to '1'.
    *
    * <p>This allows you to iterate over a bit set using a 'foreach' construct.
@@ -86,6 +104,10 @@ public final class BitSets {
     };
   }
 
+  public static Iterable<Integer> toIter(final ImmutableBitSet bitSet) {
+    return bitSet;
+  }
+
   /**
    * Converts a bitset to a list.
    *

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java b/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
index 8ad1742..20ecfa5 100644
--- a/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
+++ b/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
@@ -63,7 +63,6 @@ import java.lang.reflect.Method;
 import java.sql.ResultSet;
 import java.sql.Time;
 import java.sql.Timestamp;
-import java.util.BitSet;
 import java.util.Calendar;
 import java.util.Collection;
 import java.util.Collections;
@@ -265,14 +264,15 @@ public enum BuiltInMethod {
   ELEMENT(SqlFunctions.class, "element", List.class),
   SELECTIVITY(Selectivity.class, "getSelectivity", RexNode.class),
   UNIQUE_KEYS(UniqueKeys.class, "getUniqueKeys", boolean.class),
-  COLUMN_UNIQUENESS(ColumnUniqueness.class, "areColumnsUnique", BitSet.class,
-      boolean.class),
+  COLUMN_UNIQUENESS(ColumnUniqueness.class, "areColumnsUnique",
+      ImmutableBitSet.class, boolean.class),
   ROW_COUNT(RowCount.class, "getRowCount"),
   DISTINCT_ROW_COUNT(DistinctRowCount.class, "getDistinctRowCount",
-      BitSet.class, RexNode.class),
+      ImmutableBitSet.class, RexNode.class),
   PERCENTAGE_ORIGINAL_ROWS(PercentageOriginalRows.class,
       "getPercentageOriginalRows"),
-  POPULATION_SIZE(PopulationSize.class, "getPopulationSize", BitSet.class),
+  POPULATION_SIZE(PopulationSize.class, "getPopulationSize",
+      ImmutableBitSet.class),
   COLUMN_ORIGIN(ColumnOrigin.class, "getColumnOrigins", int.class),
   CUMULATIVE_COST(CumulativeCost.class, "getCumulativeCost"),
   NON_CUMULATIVE_COST(NonCumulativeCost.class, "getNonCumulativeCost"),

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/core/src/main/java/org/apache/calcite/util/ImmutableBitSet.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/ImmutableBitSet.java b/core/src/main/java/org/apache/calcite/util/ImmutableBitSet.java
new file mode 100644
index 0000000..a757dd5
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/util/ImmutableBitSet.java
@@ -0,0 +1,882 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.util;
+
+import org.apache.calcite.runtime.Utilities;
+
+import com.google.common.base.Function;
+import com.google.common.collect.Maps;
+
+import java.io.Serializable;
+import java.util.AbstractList;
+import java.util.Arrays;
+import java.util.BitSet;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.SortedMap;
+
+/**
+ * An immutable list of bits.
+ */
+public class ImmutableBitSet
+    implements Iterable<Integer>, Serializable, Comparable<ImmutableBitSet> {
+  /** Compares bit sets topologically, so that enclosing bit sets come first,
+   * using natural ordering to break ties. */
+  public static final Comparator<ImmutableBitSet> COMPARATOR =
+      new Comparator<ImmutableBitSet>() {
+        public int compare(ImmutableBitSet o1, ImmutableBitSet o2) {
+          if (o1.equals(o2)) {
+            return 0;
+          }
+          if (o1.contains(o2)) {
+            return -1;
+          }
+          if (o2.contains(o1)) {
+            return 1;
+          }
+          return o1.compareTo(o2);
+        }
+      };
+
+  // BitSets are packed into arrays of "words."  Currently a word is
+  // a long, which consists of 64 bits, requiring 6 address bits.
+  // The choice of word size is determined purely by performance concerns.
+  private static final int ADDRESS_BITS_PER_WORD = 6;
+  private static final int BITS_PER_WORD = 1 << ADDRESS_BITS_PER_WORD;
+
+  /* Used to shift left or right for a partial word mask */
+  private static final long WORD_MASK = 0xffffffffffffffffL;
+
+  private static final long[] EMPTY_LONGS = new long[0];
+
+  private static final ImmutableBitSet EMPTY =
+      new ImmutableBitSet(EMPTY_LONGS);
+
+  public static final Function<? super BitSet, ImmutableBitSet> FROM_BIT_SET =
+      new Function<BitSet, ImmutableBitSet>() {
+        public ImmutableBitSet apply(BitSet input) {
+          return ImmutableBitSet.of(BitSets.toIter(input));
+        }
+      };
+
+  private final long[] words;
+
+  /** Private constructor. Does not copy the array. */
+  private ImmutableBitSet(long[] words) {
+    this.words = words;
+    assert words.length == 0
+        ? words == EMPTY_LONGS
+        : words[words.length - 1] != 0L;
+  }
+
+  /** Creates an ImmutableBitSet with no bits. */
+  public static ImmutableBitSet of() {
+    return EMPTY;
+  }
+
+  public static ImmutableBitSet of(int... bits) {
+    int max = -1;
+    for (int bit : bits) {
+      max = Math.max(bit, max);
+    }
+    if (max == -1) {
+      return EMPTY;
+    }
+    long[] words = new long[wordIndex(max) + 1];
+    for (int bit : bits) {
+      int wordIndex = wordIndex(bit);
+      words[wordIndex] |= 1L << bit;
+    }
+    return new ImmutableBitSet(words);
+  }
+
+  public static ImmutableBitSet of(Iterable<Integer>  bits) {
+    if (bits instanceof ImmutableBitSet) {
+      return (ImmutableBitSet) bits;
+    }
+    int max = -1;
+    for (int bit : bits) {
+      max = Math.max(bit, max);
+    }
+    if (max == -1) {
+      return EMPTY;
+    }
+    long[] words = new long[wordIndex(max) + 1];
+    for (int bit : bits) {
+      int wordIndex = wordIndex(bit);
+      words[wordIndex] |= 1L << bit;
+    }
+    return new ImmutableBitSet(words);
+  }
+
+  /**
+   * Creates an ImmutableBitSet with given bits set.
+   *
+   * <p>For example, <code>of(ImmutableIntList.of(0, 3))</code> returns a bit
+   * set with bits {0, 3} set.
+   *
+   * @param bits Collection of bits to set
+   * @return Bit set
+   */
+  public static ImmutableBitSet of(ImmutableIntList bits) {
+    return builder().addAll(bits).build();
+  }
+
+  /**
+   * Creates an ImmutableBitSet with bits from {@code fromIndex} (inclusive) to
+   * specified {@code toIndex} (exclusive) set to {@code true}.
+   *
+   * <p>For example, {@code range(0, 3)} returns a bit set with bits
+   * {0, 1, 2} set.
+   *
+   * @param fromIndex Index of the first bit to be set.
+   * @param toIndex   Index after the last bit to be set.
+   * @return Bit set
+   */
+  public static ImmutableBitSet range(int fromIndex, int toIndex) {
+    if (fromIndex > toIndex) {
+      throw new IllegalArgumentException();
+    }
+    if (toIndex < 0) {
+      throw new IllegalArgumentException();
+    }
+    if (fromIndex == toIndex) {
+      return EMPTY;
+    }
+    int startWordIndex = wordIndex(fromIndex);
+    int endWordIndex   = wordIndex(toIndex - 1);
+    long[] words = new long[endWordIndex + 1];
+
+    long firstWordMask = WORD_MASK << fromIndex;
+    long lastWordMask  = WORD_MASK >>> -toIndex;
+    if (startWordIndex == endWordIndex) {
+      // One word
+      words[startWordIndex] |= firstWordMask & lastWordMask;
+    } else {
+      // First word, middle words, last word
+      words[startWordIndex] |= firstWordMask;
+      for (int i = startWordIndex + 1; i < endWordIndex; i++) {
+        words[i] = WORD_MASK;
+      }
+      words[endWordIndex] |= lastWordMask;
+    }
+    return new ImmutableBitSet(words);
+  }
+
+  /** Creates an ImmutableBitSet with bits between 0 and {@code toIndex} set. */
+  public static ImmutableBitSet range(int toIndex) {
+    return range(0, toIndex);
+  }
+
+  /**
+   * Given a bit index, return word index containing it.
+   */
+  private static int wordIndex(int bitIndex) {
+    return bitIndex >> ADDRESS_BITS_PER_WORD;
+  }
+
+  /**
+   * Returns the value of the bit with the specified index. The value
+   * is {@code true} if the bit with the index {@code bitIndex}
+   * is currently set in this {@code ImmutableBitSet}; otherwise, the result
+   * is {@code false}.
+   *
+   * @param  bitIndex   the bit index
+   * @return the value of the bit with the specified index
+   * @throws IndexOutOfBoundsException if the specified index is negative
+   */
+  public boolean get(int bitIndex) {
+    if (bitIndex < 0) {
+      throw new IndexOutOfBoundsException("bitIndex < 0: " + bitIndex);
+    }
+    int wordIndex = wordIndex(bitIndex);
+    return (wordIndex < words.length)
+        && ((words[wordIndex] & (1L << bitIndex)) != 0);
+  }
+
+  /**
+   * Returns a string representation of this bit set. For every index
+   * for which this {@code BitSet} contains a bit in the set
+   * state, the decimal representation of that index is included in
+   * the result. Such indices are listed in order from lowest to
+   * highest, separated by ",&nbsp;" (a comma and a space) and
+   * surrounded by braces, resulting in the usual mathematical
+   * notation for a set of integers.
+   *
+   * <p>Example:
+   * <pre>
+   * BitSet drPepper = new BitSet();</pre>
+   * Now {@code drPepper.toString()} returns "{@code {}}".
+   * <pre>
+   * drPepper.set(2);</pre>
+   * Now {@code drPepper.toString()} returns "{@code {2}}".
+   * <pre>
+   * drPepper.set(4);
+   * drPepper.set(10);</pre>
+   * Now {@code drPepper.toString()} returns "{@code {2, 4, 10}}".
+   *
+   * @return a string representation of this bit set
+   */
+  public String toString() {
+    int numBits = words.length * BITS_PER_WORD;
+    StringBuilder b = new StringBuilder(6 * numBits + 2);
+    b.append('{');
+
+    int i = nextSetBit(0);
+    if (i != -1) {
+      b.append(i);
+      for (i = nextSetBit(i + 1); i >= 0; i = nextSetBit(i + 1)) {
+        int endOfRun = nextClearBit(i);
+        do { b.append(", ").append(i); }
+        while (++i < endOfRun);
+      }
+    }
+
+    b.append('}');
+    return b.toString();
+  }
+
+  /**
+   * Returns true if the specified {@code ImmutableBitSet} has any bits set to
+   * {@code true} that are also set to {@code true} in this
+   * {@code ImmutableBitSet}.
+   *
+   * @param  set {@code ImmutableBitSet} to intersect with
+   * @return boolean indicating whether this {@code ImmutableBitSet} intersects
+   *         the specified {@code ImmutableBitSet}
+   */
+  public boolean intersects(ImmutableBitSet set) {
+    for (int i = Math.min(words.length, set.words.length) - 1; i >= 0; i--) {
+      if ((words[i] & set.words[i]) != 0) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /** Returns the number of bits set to {@code true} in this
+   * {@code ImmutableBitSet}. */
+  public int cardinality() {
+    return countBits(words);
+  }
+
+  private static int countBits(long[] words) {
+    int sum = 0;
+    for (long word : words) {
+      sum += Long.bitCount(word);
+    }
+    return sum;
+  }
+
+  /**
+   * Returns the hash code value for this bit set. The hash code
+   * depends only on which bits are set within this {@code ImmutableBitSet}.
+   *
+   * <p>The hash code is defined using the same calculation as
+   * {@link java.util.BitSet#hashCode()}.
+   *
+   * @return the hash code value for this bit set
+   */
+  public int hashCode() {
+    long h = 1234;
+    for (int i = words.length; --i >= 0;) {
+      h ^= words[i] * (i + 1);
+    }
+    return (int) ((h >> 32) ^ h);
+  }
+
+  /**
+   * Returns the number of bits of space actually in use by this
+   * {@code ImmutableBitSet} to represent bit values.
+   * The maximum element in the set is the size - 1st element.
+   *
+   * @return the number of bits currently in this bit set
+   */
+  public int size() {
+    return words.length * BITS_PER_WORD;
+  }
+
+  /**
+   * Compares this object against the specified object.
+   * The result is {@code true} if and only if the argument is
+   * not {@code null} and is a {@code ImmutableBitSet} object that has
+   * exactly the same set of bits set to {@code true} as this bit
+   * set.
+   *
+   * @param  obj the object to compare with
+   * @return {@code true} if the objects are the same;
+   *         {@code false} otherwise
+   * @see    #size()
+   */
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (!(obj instanceof ImmutableBitSet)) {
+      return false;
+    }
+    ImmutableBitSet set = (ImmutableBitSet) obj;
+    return Arrays.equals(words, set.words);
+  }
+
+  /** Compares this ImmutableBitSet with another, using a lexicographic
+   * ordering.
+   *
+   * <p>Bit sets {@code (), (0), (0, 1), (0, 1, 3), (1), (2, 3)} are in sorted
+   * order.</p>
+   */
+  public int compareTo(ImmutableBitSet o) {
+    int i = 0;
+    for (;;) {
+      int n0 = nextSetBit(i);
+      int n1 = o.nextSetBit(i);
+      int c = Utilities.compare(n0, n1);
+      if (c != 0 || n0 < 0) {
+        return c;
+      }
+      i = n0 + 1;
+    }
+  }
+
+  /**
+   * Returns the index of the first bit that is set to {@code true}
+   * that occurs on or after the specified starting index. If no such
+   * bit exists then {@code -1} is returned.
+   *
+   * <p>Based upon {@link BitSet#nextSetBit}.
+   *
+   * @param  fromIndex the index to start checking from (inclusive)
+   * @return the index of the next set bit, or {@code -1} if there
+   *         is no such bit
+   * @throws IndexOutOfBoundsException if the specified index is negative
+   */
+  public int nextSetBit(int fromIndex) {
+    if (fromIndex < 0) {
+      throw new IndexOutOfBoundsException("fromIndex < 0: " + fromIndex);
+    }
+    int u = wordIndex(fromIndex);
+    if (u >= words.length) {
+      return -1;
+    }
+    long word = words[u] & (WORD_MASK << fromIndex);
+
+    while (true) {
+      if (word != 0) {
+        return (u * BITS_PER_WORD) + Long.numberOfTrailingZeros(word);
+      }
+      if (++u == words.length) {
+        return -1;
+      }
+      word = words[u];
+    }
+  }
+
+  /**
+   * Returns the index of the first bit that is set to {@code false}
+   * that occurs on or after the specified starting index.
+   *
+   * @param  fromIndex the index to start checking from (inclusive)
+   * @return the index of the next clear bit
+   * @throws IndexOutOfBoundsException if the specified index is negative
+   */
+  public int nextClearBit(int fromIndex) {
+    if (fromIndex < 0) {
+      throw new IndexOutOfBoundsException("fromIndex < 0: " + fromIndex);
+    }
+    int u = wordIndex(fromIndex);
+    if (u >= words.length) {
+      return fromIndex;
+    }
+    long word = ~words[u] & (WORD_MASK << fromIndex);
+
+    while (true) {
+      if (word != 0) {
+        return (u * BITS_PER_WORD) + Long.numberOfTrailingZeros(word);
+      }
+      if (++u == words.length) {
+        return words.length * BITS_PER_WORD;
+      }
+      word = ~words[u];
+    }
+  }
+
+  /**
+   * Returns the index of the nearest bit that is set to {@code false}
+   * that occurs on or before the specified starting index.
+   * If no such bit exists, or if {@code -1} is given as the
+   * starting index, then {@code -1} is returned.
+   *
+   * @param  fromIndex the index to start checking from (inclusive)
+   * @return the index of the previous clear bit, or {@code -1} if there
+   *         is no such bit
+   * @throws IndexOutOfBoundsException if the specified index is less
+   *         than {@code -1}
+   */
+  public int previousClearBit(int fromIndex) {
+    if (fromIndex < 0) {
+      if (fromIndex == -1) {
+        return -1;
+      }
+      throw new IndexOutOfBoundsException("fromIndex < -1: " + fromIndex);
+    }
+
+    int u = wordIndex(fromIndex);
+    if (u >= words.length) {
+      return fromIndex;
+    }
+    long word = ~words[u] & (WORD_MASK >>> -(fromIndex + 1));
+
+    while (true) {
+      if (word != 0) {
+        return (u + 1) * BITS_PER_WORD - 1 - Long.numberOfLeadingZeros(word);
+      }
+      if (u-- == 0) {
+        return -1;
+      }
+      word = ~words[u];
+    }
+  }
+
+  public Iterator<Integer> iterator() {
+    return new Iterator<Integer>() {
+      int i = nextSetBit(0);
+
+      public boolean hasNext() {
+        return i >= 0;
+      }
+
+      public Integer next() {
+        int prev = i;
+        i = nextSetBit(i + 1);
+        return prev;
+      }
+
+      public void remove() {
+        throw new UnsupportedOperationException();
+      }
+    };
+  }
+
+  /** Converts this bit set to a list. */
+  public IntList toList() {
+    final IntList list = new IntList();
+    for (int i = nextSetBit(0); i >= 0; i = nextSetBit(i + 1)) {
+      list.add(i);
+    }
+    return list;
+  }
+
+  /** Creates a view onto this bit set as a list of integers.
+   *
+   * <p>The {@code cardinality} and {@code get} methods are both O(n), but
+   * the iterator is efficient. The list is memory efficient, and the CPU cost
+   * breaks even (versus {@link #toList}) if you intend to scan it only once. */
+  public List<Integer> asList() {
+    return new AbstractList<Integer>() {
+      @Override public Integer get(int index) {
+        return nth(index);
+      }
+
+      @Override public int size() {
+        return cardinality();
+      }
+
+      @Override public Iterator<Integer> iterator() {
+        return ImmutableBitSet.this.iterator();
+      }
+    };
+  }
+
+  /**
+   * Converts this bit set to an array.
+   *
+   * @return Array of set bits
+   */
+  public int[] toArray() {
+    final int[] integers = new int[cardinality()];
+    int j = 0;
+    for (int i = nextSetBit(0); i >= 0; i = nextSetBit(i + 1)) {
+      integers[j++] = i;
+    }
+    return integers;
+  }
+
+  /** Returns the union of this bit set with another. */
+  public ImmutableBitSet union(ImmutableBitSet other) {
+    return builder(this)
+        .addAll(other)
+        .build();
+  }
+
+  /** Returns the union of a number of bit sets. */
+  public static ImmutableBitSet union(
+      Iterable<? extends ImmutableBitSet> sets) {
+    final Builder builder = builder();
+    for (ImmutableBitSet set : sets) {
+      builder.addAll(set);
+    }
+    return builder.build();
+  }
+
+  /** Returns a bit set with all the bits in this set that are not in
+   * another.
+   *
+   *  @see BitSet#andNot(java.util.BitSet) */
+  public ImmutableBitSet except(ImmutableBitSet that) {
+    final Builder builder = builder(this);
+    builder.removeAll(that);
+    return builder.build(this);
+  }
+
+  /** Returns a bit set with all the bits set in both this set and in
+   *  another.
+   *
+   *  @see BitSet#and */
+  public ImmutableBitSet intersect(ImmutableBitSet that) {
+    final Builder builder = builder(this);
+    builder.intersect(that);
+    return builder.build(this);
+  }
+
+  /**
+   * Returns true if all bits set in the second parameter are also set in the
+   * first. In other words, whether x is a super-set of y.
+   *
+   * @param set1 Bitmap to be checked
+   *
+   * @return Whether all bits in set1 are set in set0
+   */
+  public boolean contains(ImmutableBitSet set1) {
+    for (int i = set1.nextSetBit(0); i >= 0; i = set1.nextSetBit(i + 1)) {
+      if (!get(i)) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  /**
+   * The ordinal of a given bit, or -1 if it is not set.
+   */
+  public int indexOf(int bit) {
+    for (int i = nextSetBit(0), k = 0;; i = nextSetBit(i + 1), ++k) {
+      if (i < 0) {
+        return -1;
+      }
+      if (i == bit) {
+        return k;
+      }
+    }
+  }
+
+  /** Computes the closure of a map from integers to bits.
+   *
+   * <p>The input must have an entry for each position.
+   *
+   * <p>Does not modify the input map or its bit sets. */
+  public static SortedMap<Integer, ImmutableBitSet> closure(
+      SortedMap<Integer, ImmutableBitSet> equivalence) {
+    final Closure closure = new Closure(equivalence);
+    return closure.closure;
+  }
+
+  /**
+   * Returns the "logical size" of this {@code ImmutableBitSet}: the index of
+   * the highest set bit in the {@code ImmutableBitSet} plus one. Returns zero
+   * if the {@code ImmutableBitSet} contains no set bits.
+   *
+   * @return the logical size of this {@code ImmutableBitSet}
+   */
+  public int length() {
+    if (words.length == 0) {
+      return 0;
+    }
+    return BITS_PER_WORD * (words.length - 1)
+        + (BITS_PER_WORD - Long.numberOfLeadingZeros(words[words.length - 1]));
+  }
+
+  /**
+   * Returns true if this {@code ImmutableBitSet} contains no bits that are set
+   * to {@code true}.
+   */
+  public boolean isEmpty() {
+    return words.length == 0;
+  }
+
+  public static Builder builder() {
+    return new Builder();
+  }
+
+  public static Builder builder(ImmutableBitSet bitSet) {
+    return new Builder(bitSet);
+  }
+
+  /** Returns the {@code n}th set bit.
+   *
+   * @throws java.lang.IndexOutOfBoundsException if n is less than 0 or greater
+   * than the number of bits set */
+  public int nth(int n) {
+    int start = 0;
+    for (long word : words) {
+      final int bitCount = Long.bitCount(word);
+      if (n < bitCount) {
+        while (word != 0) {
+          if ((word & 1) == 1) {
+            if (n == 0) {
+              return start;
+            }
+            --n;
+          }
+          word >>= 1;
+          ++start;
+        }
+      }
+      start += 64;
+      n -= bitCount;
+    }
+    throw new IndexOutOfBoundsException("index out of range: " + n);
+  }
+
+  /** Returns a bit set the same as this but with a given bit cleared. */
+  public ImmutableBitSet clear(int i) {
+    return except(ImmutableBitSet.of(i));
+  }
+
+  /** Returns a {@link BitSet} that has the same contents as this
+   * {@code ImmutableBitSet}. */
+  public BitSet toBitSet() {
+    return BitSets.of(this);
+  }
+
+  /**
+   * Setup equivalence Sets for each position. If i & j are equivalent then
+   * they will have the same equivalence Set. The algorithm computes the
+   * closure relation at each position for the position wrt to positions
+   * greater than it. Once a closure is computed for a position, the closure
+   * Set is set on all its descendants. So the closure computation bubbles up
+   * from lower positions and the final equivalence Set is propagated down
+   * from the lowest element in the Set.
+   */
+  private static class Closure {
+    private SortedMap<Integer, ImmutableBitSet> equivalence;
+    private final SortedMap<Integer, ImmutableBitSet> closure =
+        Maps.newTreeMap();
+
+    public Closure(SortedMap<Integer, ImmutableBitSet> equivalence) {
+      this.equivalence = equivalence;
+      final ImmutableIntList keys =
+          ImmutableIntList.copyOf(equivalence.keySet());
+      for (int pos : keys) {
+        computeClosure(pos);
+      }
+    }
+
+    private ImmutableBitSet computeClosure(int pos) {
+      ImmutableBitSet o = closure.get(pos);
+      if (o != null) {
+        return o;
+      }
+      final ImmutableBitSet b = equivalence.get(pos);
+      o = b;
+      int i = b.nextSetBit(pos + 1);
+      for (; i >= 0; i = b.nextSetBit(i + 1)) {
+        o = o.union(computeClosure(i));
+      }
+      closure.put(pos, o);
+      i = o.nextSetBit(pos + 1);
+      for (; i >= 0; i = b.nextSetBit(i + 1)) {
+        closure.put(i, o);
+      }
+      return o;
+    }
+  }
+
+  /** Builder. */
+  public static class Builder {
+    private long[] words;
+
+    public Builder(ImmutableBitSet bitSet) {
+      words = bitSet.words.clone();
+    }
+
+    public Builder() {
+      words = EMPTY_LONGS;
+    }
+
+    /** Builds an ImmutableBitSet from the contents of this Builder.
+     *
+     * <p>After calling this method, the Builder cannot be used again. */
+    public ImmutableBitSet build() {
+      if (words.length == 0) {
+        return EMPTY;
+      }
+      long[] words = this.words;
+      this.words = null; // prevent re-use of builder
+      return new ImmutableBitSet(words);
+    }
+
+    /** Builds an ImmutableBitSet from the contents of this Builder, using
+     * an existing ImmutableBitSet if it happens to have the same contents.
+     *
+     * <p>Supplying the existing bit set if useful for set operations,
+     * where there is a significant chance that the original bit set is
+     * unchanged. We save memory because we use the same copy. For example:
+     *
+     * <blockquote><pre>
+     * ImmutableBitSet primeNumbers;
+     * ImmutableBitSet hundreds = ImmutableBitSet.of(100, 200, 300);
+     * return primeNumbers.except(hundreds);</pre></blockquote>
+     *
+     * <p>After calling this method, the Builder cannot be used again. */
+    public ImmutableBitSet build(ImmutableBitSet bitSet) {
+      if (wouldEqual(bitSet)) {
+        return bitSet;
+      }
+      return build();
+    }
+
+    public Builder set(int bit) {
+      if (words == null) {
+        throw new IllegalArgumentException("can only use builder once");
+      }
+      int wordIndex = wordIndex(bit);
+      if (wordIndex >= words.length) {
+        words = Arrays.copyOf(words, wordIndex + 1);
+      }
+      words[wordIndex] |= 1L << bit;
+      return this;
+    }
+
+    private void trim(int wordCount) {
+      while (wordCount > 0 && words[wordCount - 1] == 0L) {
+        --wordCount;
+      }
+      if (wordCount == words.length) {
+        return;
+      }
+      if (wordCount == 0) {
+        words = EMPTY_LONGS;
+      } else {
+        words = Arrays.copyOfRange(words, 0, wordCount);
+      }
+    }
+
+    public Builder clear(int bit) {
+      int wordIndex = wordIndex(bit);
+      if (wordIndex < words.length) {
+        words[wordIndex] &= ~(1L << bit);
+        trim(words.length);
+      }
+      return this;
+    }
+
+    /** Returns whether the bit set that would be created by this Builder would
+     * equal a given bit set. */
+    public boolean wouldEqual(ImmutableBitSet bitSet) {
+      if (words == null) {
+        throw new IllegalArgumentException("can only use builder once");
+      }
+      return Arrays.equals(words, bitSet.words);
+    }
+
+    /** Returns the number of set bits. */
+    public int cardinality() {
+      return countBits(words);
+    }
+
+    /** Sets all bits in a given bit set. */
+    public Builder addAll(ImmutableBitSet bitSet) {
+      for (Integer bit : bitSet) {
+        set(bit);
+      }
+      return this;
+    }
+
+    /** Sets all bits in a given list of bits. */
+    public Builder addAll(Iterable<Integer> integers) {
+      for (Integer integer : integers) {
+        set(integer);
+      }
+      return this;
+    }
+
+    /** Sets all bits in a given list of {@code int}s. */
+    public Builder addAll(ImmutableIntList integers) {
+      //noinspection ForLoopReplaceableByForEach
+      for (int i = 0; i < integers.size(); i++) {
+        set(integers.get(i));
+      }
+      return this;
+    }
+
+    /** Clears all bits in a given bit set. */
+    public Builder removeAll(ImmutableBitSet bitSet) {
+      for (Integer bit : bitSet) {
+        clear(bit);
+      }
+      return this;
+    }
+
+    /** Sets a range of bits, from {@code from} to {@code to} - 1. */
+    public Builder set(int fromIndex, int toIndex) {
+      if (fromIndex > toIndex) {
+        throw new IllegalArgumentException();
+      }
+      if (toIndex < 0) {
+        throw new IllegalArgumentException();
+      }
+      if (fromIndex < toIndex) {
+        // Increase capacity if necessary
+        int startWordIndex = wordIndex(fromIndex);
+        int endWordIndex   = wordIndex(toIndex - 1);
+        if (endWordIndex >= words.length) {
+          words = Arrays.copyOf(words, endWordIndex + 1);
+        }
+
+        long firstWordMask = WORD_MASK << fromIndex;
+        long lastWordMask  = WORD_MASK >>> -toIndex;
+        if (startWordIndex == endWordIndex) {
+          // One word
+          words[startWordIndex] |= firstWordMask & lastWordMask;
+        } else {
+          // First word, middle words, last word
+          words[startWordIndex] |= firstWordMask;
+          for (int i = startWordIndex + 1; i < endWordIndex; i++) {
+            words[i] = WORD_MASK;
+          }
+          words[endWordIndex] |= lastWordMask;
+        }
+      }
+      return this;
+    }
+
+    public boolean isEmpty() {
+      return words.length == 0;
+    }
+
+    public void intersect(ImmutableBitSet that) {
+      int x = Math.min(words.length, that.words.length);
+      for (int i = 0; i < x; i++) {
+        words[i] &= that.words[i];
+      }
+      trim(x);
+    }
+  }
+}
+
+// End ImmutableBitSet.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/core/src/main/java/org/apache/calcite/util/mapping/Mappings.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/mapping/Mappings.java b/core/src/main/java/org/apache/calcite/util/mapping/Mappings.java
index 8d9733b..8c16163 100644
--- a/core/src/main/java/org/apache/calcite/util/mapping/Mappings.java
+++ b/core/src/main/java/org/apache/calcite/util/mapping/Mappings.java
@@ -17,6 +17,7 @@
 package org.apache.calcite.util.mapping;
 
 import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.IntList;
 import org.apache.calcite.util.Permutation;
 import org.apache.calcite.util.Util;
@@ -188,6 +189,28 @@ public abstract class Mappings {
   }
 
   /**
+   * Applies a mapping to an {@code ImmutableBitSet}.
+   *
+   * <p>If the mapping does not affect the bit set, returns the original.
+   * Never changes the original.
+   *
+   * @param mapping Mapping
+   * @param bitSet  Bit set
+   * @return Bit set with mapping applied
+   */
+  public static ImmutableBitSet apply(Mapping mapping, ImmutableBitSet bitSet) {
+    final ImmutableBitSet.Builder builder = ImmutableBitSet.builder();
+    for (int source : bitSet) {
+      final int target = mapping.getTarget(source);
+      builder.set(target);
+    }
+    if (builder.wouldEqual(bitSet)) {
+      return bitSet;
+    }
+    return builder.build();
+  }
+
+  /**
    * Applies a mapping to a list.
    *
    * @param mapping Mapping

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/core/src/test/java/org/apache/calcite/plan/RelWriterTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/plan/RelWriterTest.java b/core/src/test/java/org/apache/calcite/plan/RelWriterTest.java
index 5ef1877..6dc6aa4 100644
--- a/core/src/test/java/org/apache/calcite/plan/RelWriterTest.java
+++ b/core/src/test/java/org/apache/calcite/plan/RelWriterTest.java
@@ -32,7 +32,7 @@ import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.test.JdbcTest;
 import org.apache.calcite.tools.Frameworks;
-import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
@@ -138,7 +138,7 @@ public class RelWriterTest {
                 final RelDataType bigIntType =
                     cluster.getTypeFactory().createSqlType(SqlTypeName.BIGINT);
                 LogicalAggregate aggregate =
-                    new LogicalAggregate(cluster, filter, BitSets.of(0),
+                    new LogicalAggregate(cluster, filter, ImmutableBitSet.of(0),
                         ImmutableList.of(
                             new AggregateCall(SqlStdOperatorTable.COUNT,
                                 true, ImmutableList.of(1), bigIntType, "c"),

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/core/src/test/java/org/apache/calcite/test/CalciteSuite.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/CalciteSuite.java b/core/src/test/java/org/apache/calcite/test/CalciteSuite.java
index 3a3ec53..081ac93 100644
--- a/core/src/test/java/org/apache/calcite/test/CalciteSuite.java
+++ b/core/src/test/java/org/apache/calcite/test/CalciteSuite.java
@@ -33,6 +33,7 @@ import org.apache.calcite.tools.FrameworksTest;
 import org.apache.calcite.tools.PlannerTest;
 import org.apache.calcite.util.BitSetsTest;
 import org.apache.calcite.util.ChunkListTest;
+import org.apache.calcite.util.ImmutableBitSetTest;
 import org.apache.calcite.util.PartiallyOrderedSetTest;
 import org.apache.calcite.util.PermutationTestCase;
 import org.apache.calcite.util.ReflectVisitorTest;
@@ -57,6 +58,7 @@ import org.junit.runners.Suite;
     // very fast tests (under 0.1s)
     ArrayTableTest.class,
     BitSetsTest.class,
+    ImmutableBitSetTest.class,
     DirectedGraphTest.class,
     ReflectVisitorTest.class,
     RelOptUtilTest.class,

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/core/src/test/java/org/apache/calcite/test/MockCatalogReader.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/MockCatalogReader.java b/core/src/test/java/org/apache/calcite/test/MockCatalogReader.java
index 7abe734..f915e1b 100644
--- a/core/src/test/java/org/apache/calcite/test/MockCatalogReader.java
+++ b/core/src/test/java/org/apache/calcite/test/MockCatalogReader.java
@@ -41,6 +41,7 @@ import org.apache.calcite.sql.validate.SqlMonikerType;
 import org.apache.calcite.sql.validate.SqlMonotonicity;
 import org.apache.calcite.sql.validate.SqlValidatorCatalogReader;
 import org.apache.calcite.sql.validate.SqlValidatorUtil;
+import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Util;
 
@@ -49,7 +50,6 @@ import com.google.common.collect.Ordering;
 
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.BitSet;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
@@ -401,7 +401,7 @@ public class MockCatalogReader implements Prepare.CatalogReader {
       return collationList;
     }
 
-    public boolean isKey(BitSet columns) {
+    public boolean isKey(ImmutableBitSet columns) {
       return false;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/core/src/test/java/org/apache/calcite/test/RelMetadataTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/RelMetadataTest.java b/core/src/test/java/org/apache/calcite/test/RelMetadataTest.java
index 41caea0..67b30de 100644
--- a/core/src/test/java/org/apache/calcite/test/RelMetadataTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RelMetadataTest.java
@@ -30,6 +30,7 @@ import org.apache.calcite.rel.metadata.ReflectiveRelMetadataProvider;
 import org.apache.calcite.rel.metadata.RelColumnOrigin;
 import org.apache.calcite.rel.metadata.RelMetadataProvider;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.util.ImmutableBitSet;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterables;
@@ -41,7 +42,6 @@ import org.junit.Test;
 
 import java.lang.reflect.Method;
 import java.util.ArrayList;
-import java.util.BitSet;
 import java.util.List;
 import java.util.Set;
 
@@ -539,7 +539,7 @@ public class RelMetadataTest extends SqlToRelTestBase {
   @Test public void testDistinctRowCountTable() {
     // no unique key information is available so return null
     RelNode rel = convertSql("select * from emp where deptno = 10");
-    BitSet groupKey = new BitSet();
+    ImmutableBitSet groupKey = ImmutableBitSet.of();
     Double result =
         RelMetadataQuery.getDistinctRowCount(
             rel, groupKey, null);

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/core/src/test/java/org/apache/calcite/test/RexProgramTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/RexProgramTest.java b/core/src/test/java/org/apache/calcite/test/RexProgramTest.java
index d1d0bae..aaa6b01 100644
--- a/core/src/test/java/org/apache/calcite/test/RexProgramTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RexProgramTest.java
@@ -34,7 +34,7 @@ import org.apache.calcite.rex.RexProgramBuilder;
 import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.type.SqlTypeName;
-import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.TestUtil;
 import org.apache.calcite.util.Util;
 
@@ -46,7 +46,6 @@ import org.junit.Test;
 
 import java.math.BigDecimal;
 import java.util.Arrays;
-import java.util.BitSet;
 import java.util.List;
 
 import static org.hamcrest.CoreMatchers.equalTo;
@@ -322,7 +321,7 @@ public class RexProgramTest {
     return builder;
   }
 
-  static boolean strongIf(RexNode e, BitSet b) {
+  static boolean strongIf(RexNode e, ImmutableBitSet b) {
     return Strong.is(e, b);
   }
 
@@ -330,11 +329,11 @@ public class RexProgramTest {
   @Test public void testStrong() {
     final RelDataType intType = typeFactory.createSqlType(SqlTypeName.INTEGER);
 
-    final BitSet c = BitSets.of();
-    final BitSet c0 = BitSets.of(0);
-    final BitSet c1 = BitSets.of(1);
-    final BitSet c01 = BitSets.of(0, 1);
-    final BitSet c13 = BitSets.of(1, 3);
+    final ImmutableBitSet c = ImmutableBitSet.of();
+    final ImmutableBitSet c0 = ImmutableBitSet.of(0);
+    final ImmutableBitSet c1 = ImmutableBitSet.of(1);
+    final ImmutableBitSet c01 = ImmutableBitSet.of(0, 1);
+    final ImmutableBitSet c13 = ImmutableBitSet.of(1, 3);
 
     // input ref
     final RexInputRef aRef = rexBuilder.makeInputRef(intType, 0);

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/core/src/test/java/org/apache/calcite/test/SqlToRelTestBase.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/SqlToRelTestBase.java b/core/src/test/java/org/apache/calcite/test/SqlToRelTestBase.java
index ab2cd05..1d88316 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlToRelTestBase.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlToRelTestBase.java
@@ -47,6 +47,7 @@ import org.apache.calcite.sql.validate.SqlValidatorTable;
 import org.apache.calcite.sql2rel.RelFieldTrimmer;
 import org.apache.calcite.sql2rel.SqlToRelConverter;
 import org.apache.calcite.sql2rel.StandardConvertletTable;
+import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.Util;
 
 import com.google.common.base.Function;
@@ -54,7 +55,6 @@ import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterables;
 
 import java.util.ArrayList;
-import java.util.BitSet;
 import java.util.List;
 
 import static org.junit.Assert.assertEquals;
@@ -353,7 +353,7 @@ public abstract class SqlToRelTestBase {
         return collationList;
       }
 
-      public boolean isKey(BitSet columns) {
+      public boolean isKey(ImmutableBitSet columns) {
         return false;
       }
 
@@ -406,7 +406,7 @@ public abstract class SqlToRelTestBase {
       return parent.getCollationList();
     }
 
-    public boolean isKey(BitSet columns) {
+    public boolean isKey(ImmutableBitSet columns) {
       return parent.isKey(columns);
     }
   }


[30/58] [abbrv] [partial] incubator-calcite git commit: [CALCITE-306] Standardize code style for "import package.*; "

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/rules/package-info.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/package-info.java b/core/src/main/java/org/apache/calcite/rel/rules/package-info.java
index b6f07af..47ef478 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/package-info.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/package-info.java
@@ -16,21 +16,37 @@
  */
 
 /**
- * Defines relational expressions.<p>&nbsp;</p>
+ * Provides a core set of planner rules.
+ *
+ * <p>Consider this package to be the "standard library" of planner rules.
+ * Most of the common rewrites that you would want to perform on logical
+ * relational expressions, or generically on any data source, are present,
+ * and have been well tested.
+ *
+ * <p>Of course, the library is never complete, and contributions are welcome.
+ *
+ * <p>Not present are rules specific to a particular data source: look in that
+ * data source's adapter.
+ *
+ * <p>Also out of the scope of this package are rules that support a particular
+ * operation, such as decorrelation or recognizing materialized views. Those are
+ * defined along with the algorithm.
+ *
+ * <p>For
  *
  * <h2>Related packages and classes</h2>
  * <ul>
  *    <li>Package<code> <a href="../sql/package-summary.html">
- *        org.eigenbase.sql</a></code>
+ *        org.apache.calcite.sql</a></code>
  *        is an object model for SQL expressions</li>
- *    <li>Package<code> <a href="../sql/package-summary.html">
- *        org.eigenbase.rex</a></code>
+ *    <li>Package<code> <a href="../rex/package-summary.html">
+ *        org.apache.calcite.rex</a></code>
  *        is an object model for relational row expressions</li>
- *    <li>Package<code> <a href="../relopt/package-summary.html">
- *        org.eigenbase.relopt</a></code>
+ *    <li>Package<code> <a href="../plan/package-summary.html">
+ *        org.apache.calcite.plan</a></code>
  *        provides an optimizer interface.</li>
  * </ul>
  */
-package org.eigenbase.rel.rules;
+package org.apache.calcite.rel.rules;
 
 // End package-info.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/type/RelCrossType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/type/RelCrossType.java b/core/src/main/java/org/apache/calcite/rel/type/RelCrossType.java
index 4418d43..d361b7b 100644
--- a/core/src/main/java/org/apache/calcite/rel/type/RelCrossType.java
+++ b/core/src/main/java/org/apache/calcite/rel/type/RelCrossType.java
@@ -14,19 +14,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.reltype;
+package org.apache.calcite.rel.type;
 
-import java.util.List;
-
-import net.hydromatic.linq4j.Ord;
+import org.apache.calcite.linq4j.Ord;
 
 import com.google.common.collect.ImmutableList;
 
+import java.util.List;
+
 /**
  * Type of the cartesian product of two or more sets of records.
  *
- * <p>Its fields are those of its constituent records, but unlike a {@link
- * RelRecordType}, those fields' names are not necessarily distinct.</p>
+ * <p>Its fields are those of its constituent records, but unlike a
+ * {@link RelRecordType}, those fields' names are not necessarily distinct.</p>
  */
 public class RelCrossType extends RelDataTypeImpl {
   //~ Instance fields --------------------------------------------------------
@@ -54,13 +54,11 @@ public class RelCrossType extends RelDataTypeImpl {
 
   //~ Methods ----------------------------------------------------------------
 
-  @Override
-  public boolean isStruct() {
+  @Override public boolean isStruct() {
     return false;
   }
 
-  @Override
-  public List<RelDataTypeField> getFieldList() {
+  @Override public List<RelDataTypeField> getFieldList() {
     return fieldList;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/type/RelDataType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/type/RelDataType.java b/core/src/main/java/org/apache/calcite/rel/type/RelDataType.java
index a470296..f378d48 100644
--- a/core/src/main/java/org/apache/calcite/rel/type/RelDataType.java
+++ b/core/src/main/java/org/apache/calcite/rel/type/RelDataType.java
@@ -14,13 +14,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.reltype;
+package org.apache.calcite.rel.type;
 
-import java.nio.charset.*;
-import java.util.*;
+import org.apache.calcite.sql.SqlCollation;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlIntervalQualifier;
+import org.apache.calcite.sql.type.SqlTypeName;
 
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.type.*;
+import java.nio.charset.Charset;
+import java.util.List;
 
 /**
  * RelDataType represents the type of a scalar expression or entire row returned
@@ -179,9 +181,9 @@ public interface RelDataType /*extends Type*/ {
 
   /**
    * Gets the {@link SqlIdentifier} associated with this type. For a
-   * predefined type, this is a simple identifier based on {@link
-   * #getSqlTypeName}. For a user-defined type, this is a compound identifier
-   * which uniquely names the type.
+   * predefined type, this is a simple identifier based on
+   * {@link #getSqlTypeName}. For a user-defined type, this is a compound
+   * identifier which uniquely names the type.
    *
    * @return SqlIdentifier, or null if this is not an SQL type
    */

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeComparability.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeComparability.java b/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeComparability.java
index 3246cb0..7dbabbb 100644
--- a/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeComparability.java
+++ b/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeComparability.java
@@ -14,9 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.reltype;
+package org.apache.calcite.rel.type;
 
-import org.eigenbase.util.*;
+import org.apache.calcite.util.Util;
 
 /**
  * RelDataTypeComparability is an enumeration of the categories of comparison

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeFactory.java b/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeFactory.java
index 0d9fd06..16520a7 100644
--- a/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeFactory.java
+++ b/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeFactory.java
@@ -14,14 +14,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.reltype;
+package org.apache.calcite.rel.type;
 
-import java.nio.charset.*;
+import org.apache.calcite.sql.SqlCollation;
+import org.apache.calcite.sql.SqlIntervalQualifier;
+import org.apache.calcite.sql.type.SqlTypeName;
 
-import java.util.*;
-
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.type.*;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
 
 /**
  * RelDataTypeFactory is a factory for datatype descriptors. It defines methods
@@ -30,7 +32,7 @@ import org.eigenbase.sql.type.*;
  * SQL 2003 is implementation defined or impractical.
  *
  * <p>This interface is an example of the
- * {@link org.eigenbase.util.Glossary#ABSTRACT_FACTORY_PATTERN abstract factory pattern}.
+ * {@link org.apache.calcite.util.Glossary#ABSTRACT_FACTORY_PATTERN abstract factory pattern}.
  * Any implementation of <code>RelDataTypeFactory</code> must ensure that type
  * objects are canonical: two types are equal if and only if they are
  * represented by the same Java object. This reduces memory consumption and
@@ -276,7 +278,7 @@ public interface RelDataTypeFactory {
 
   /**
    * Creates a
-   * {@link org.eigenbase.reltype.RelDataTypeFactory.FieldInfoBuilder}.
+   * {@link org.apache.calcite.rel.type.RelDataTypeFactory.FieldInfoBuilder}.
    */
   FieldInfoBuilder builder();
 
@@ -351,7 +353,7 @@ public interface RelDataTypeFactory {
 
     /**
      * Adds a field with a type created using
-     * {@link org.eigenbase.reltype.RelDataTypeFactory#createSqlType(org.eigenbase.sql.type.SqlTypeName)}.
+     * {@link org.apache.calcite.rel.type.RelDataTypeFactory#createSqlType(org.apache.calcite.sql.type.SqlTypeName)}.
      */
     public FieldInfoBuilder add(String name, SqlTypeName typeName) {
       add(name, typeFactory.createSqlType(typeName));
@@ -360,7 +362,7 @@ public interface RelDataTypeFactory {
 
     /**
      * Adds a field with a type created using
-     * {@link org.eigenbase.reltype.RelDataTypeFactory#createSqlType(org.eigenbase.sql.type.SqlTypeName, int)}.
+     * {@link org.apache.calcite.rel.type.RelDataTypeFactory#createSqlType(org.apache.calcite.sql.type.SqlTypeName, int)}.
      */
     public FieldInfoBuilder add(
         String name, SqlTypeName typeName, int precision) {
@@ -370,7 +372,7 @@ public interface RelDataTypeFactory {
 
     /**
      * Adds a field with a type created using
-     * {@link org.eigenbase.reltype.RelDataTypeFactory#createSqlType(org.eigenbase.sql.type.SqlTypeName, int, int)}.
+     * {@link org.apache.calcite.rel.type.RelDataTypeFactory#createSqlType(org.apache.calcite.sql.type.SqlTypeName, int, int)}.
      */
     public FieldInfoBuilder add(
         String name, SqlTypeName typeName, int precision, int scale) {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeFactoryImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeFactoryImpl.java b/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeFactoryImpl.java
index 541bd83..06ec977 100644
--- a/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeFactoryImpl.java
+++ b/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeFactoryImpl.java
@@ -14,24 +14,35 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.reltype;
-
-import java.lang.reflect.*;
-import java.nio.charset.*;
-import java.sql.*;
-import java.util.*;
-
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.util.*;
-
-import net.hydromatic.linq4j.expressions.Primitive;
+package org.apache.calcite.rel.type;
+
+import org.apache.calcite.linq4j.tree.Primitive;
+import org.apache.calcite.sql.SqlCollation;
+import org.apache.calcite.sql.type.JavaToSqlTypeConversionRules;
+import org.apache.calcite.sql.type.SqlTypeFamily;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.type.SqlTypeUtil;
+import org.apache.calcite.util.Bug;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
 
 import com.google.common.base.Preconditions;
-import com.google.common.cache.*;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 
+import java.lang.reflect.Field;
+import java.lang.reflect.Modifier;
+import java.nio.charset.Charset;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.util.AbstractList;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
 /**
  * Abstract base for implementations of {@link RelDataTypeFactory}.
  */
@@ -46,8 +57,7 @@ public abstract class RelDataTypeFactoryImpl implements RelDataTypeFactory {
           .softValues()
           .build(
               new CacheLoader<Object, RelDataType>() {
-                @Override
-                public RelDataType load(Object key) {
+                @Override public RelDataType load(Object key) {
                   if (key instanceof RelDataType) {
                     return (RelDataType) key;
                   }
@@ -147,24 +157,20 @@ public abstract class RelDataTypeFactoryImpl implements RelDataTypeFactory {
       final RelDataTypeFactory.FieldInfo fieldInfo) {
     return canonize(
         new AbstractList<String>() {
-          @Override
-          public String get(int index) {
+          @Override public String get(int index) {
             return fieldInfo.getFieldName(index);
           }
 
-          @Override
-          public int size() {
+          @Override public int size() {
             return fieldInfo.getFieldCount();
           }
         },
         new AbstractList<RelDataType>() {
-          @Override
-          public RelDataType get(int index) {
+          @Override public RelDataType get(int index) {
             return fieldInfo.getFieldType(index);
           }
 
-          @Override
-          public int size() {
+          @Override public int size() {
             return fieldInfo.getFieldCount();
           }
         });
@@ -613,8 +619,7 @@ public abstract class RelDataTypeFactoryImpl implements RelDataTypeFactory {
       return nullable;
     }
 
-    @Override
-    public RelDataTypeFamily getFamily() {
+    @Override public RelDataTypeFamily getFamily() {
       RelDataTypeFamily family = CLASS_FAMILIES.get(clazz);
       return family != null ? family : this;
     }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeFamily.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeFamily.java b/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeFamily.java
index e69544d..54a3da0 100644
--- a/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeFamily.java
+++ b/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeFamily.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.reltype;
+package org.apache.calcite.rel.type;
 
 /**
  * RelDataTypeFamily represents a family of related types. The specific criteria

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeField.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeField.java b/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeField.java
index c776c9f..2094d48 100644
--- a/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeField.java
+++ b/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeField.java
@@ -14,13 +14,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.reltype;
+package org.apache.calcite.rel.type;
 
 import java.util.Map;
 
 /**
- * RelDataTypeField represents the definition of a field in a structured {@link
- * RelDataType}.
+ * RelDataTypeField represents the definition of a field in a structured
+ * {@link RelDataType}.
  *
  * <p>Extends the {@link java.util.Map.Entry} interface to allow convenient
  * inter-operation with Java collections classes. In any implementation of this

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeFieldImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeFieldImpl.java b/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeFieldImpl.java
index 7ac2775..59eea0c 100644
--- a/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeFieldImpl.java
+++ b/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeFieldImpl.java
@@ -14,9 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.reltype;
+package org.apache.calcite.rel.type;
 
-import java.io.*;
+import java.io.Serializable;
 
 /**
  * Default implementation of {@link RelDataTypeField}.
@@ -46,15 +46,13 @@ public class RelDataTypeFieldImpl implements RelDataTypeField, Serializable {
 
   //~ Methods ----------------------------------------------------------------
 
-  @Override
-  public int hashCode() {
+  @Override public int hashCode() {
     return index
         ^ name.hashCode()
         ^ type.hashCode();
   }
 
-  @Override
-  public boolean equals(Object obj) {
+  @Override public boolean equals(Object obj) {
     if (this == obj) {
       return true;
     }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeImpl.java b/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeImpl.java
index 688666a..d917667 100644
--- a/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeImpl.java
+++ b/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeImpl.java
@@ -14,21 +14,24 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.reltype;
+package org.apache.calcite.rel.type;
 
-import java.io.*;
-import java.nio.charset.*;
-import java.util.*;
-
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.util.Pair;
-import org.eigenbase.util.Util;
+import org.apache.calcite.sql.SqlCollation;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlIntervalQualifier;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.BasicSqlType;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterables;
 
+import java.io.Serializable;
+import java.nio.charset.Charset;
+import java.util.List;
+
 /**
  * RelDataTypeImpl is an abstract base for implementations of
  * {@link RelDataType}.
@@ -60,8 +63,8 @@ public abstract class RelDataTypeImpl
   }
 
   /**
-   * Default constructor, to allow derived classes such as {@link
-   * BasicSqlType} to be {@link Serializable}.
+   * Default constructor, to allow derived classes such as
+   * {@link BasicSqlType} to be {@link Serializable}.
    *
    * <p>(The serialization specification says that a class can be serializable
    * even if its base class is not serializable, provided that the base class
@@ -269,8 +272,8 @@ public abstract class RelDataTypeImpl
     };
   }
 
-  /** Returns a {@link org.eigenbase.reltype.RelProtoDataType} that will create
-   * a type {@code typeName}.
+  /** Returns a {@link org.apache.calcite.rel.type.RelProtoDataType}
+   * that will create a type {@code typeName}.
    *
    * <p>For example, {@code proto(SqlTypeName.DATE), false}
    * will create {@code DATE NOT NULL}.</p>
@@ -290,8 +293,8 @@ public abstract class RelDataTypeImpl
     };
   }
 
-  /** Returns a {@link org.eigenbase.reltype.RelProtoDataType} that will create
-   * a type {@code typeName(precision)}.
+  /** Returns a {@link org.apache.calcite.rel.type.RelProtoDataType}
+   * that will create a type {@code typeName(precision)}.
    *
    * <p>For example, {@code proto(SqlTypeName.VARCHAR, 100, false)}
    * will create {@code VARCHAR(100) NOT NULL}.</p>
@@ -312,8 +315,8 @@ public abstract class RelDataTypeImpl
     };
   }
 
-  /** Returns a {@link org.eigenbase.reltype.RelProtoDataType} that will create
-   * a type {@code typeName(precision, scale)}.
+  /** Returns a {@link org.apache.calcite.rel.type.RelProtoDataType}
+   * that will create a type {@code typeName(precision, scale)}.
    *
    * <p>For example, {@code proto(SqlTypeName.DECIMAL, 7, 2, false)}
    * will create {@code DECIMAL(7, 2) NOT NULL}.</p>

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/type/RelDataTypePrecedenceList.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/type/RelDataTypePrecedenceList.java b/core/src/main/java/org/apache/calcite/rel/type/RelDataTypePrecedenceList.java
index 8d1f006..e2ecea1 100644
--- a/core/src/main/java/org/apache/calcite/rel/type/RelDataTypePrecedenceList.java
+++ b/core/src/main/java/org/apache/calcite/rel/type/RelDataTypePrecedenceList.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.reltype;
+package org.apache.calcite.rel.type;
 
 /**
  * RelDataTypePrecedenceList defines a type precedence list for a particular

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeSystem.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeSystem.java b/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeSystem.java
index e72722f..737a644 100644
--- a/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeSystem.java
+++ b/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeSystem.java
@@ -14,9 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.reltype;
+package org.apache.calcite.rel.type;
 
-import org.eigenbase.sql.type.SqlTypeName;
+import org.apache.calcite.sql.type.SqlTypeName;
 
 /**
  * Type system.
@@ -57,4 +57,4 @@ public interface RelDataTypeSystem {
   int getMaxNumericPrecision();
 }
 
-// End RelDataTypeFactory.java
+// End RelDataTypeSystem.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeSystemImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeSystemImpl.java b/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeSystemImpl.java
index 0f5ff54..ea639c0 100644
--- a/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeSystemImpl.java
+++ b/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeSystemImpl.java
@@ -14,11 +14,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.reltype;
+package org.apache.calcite.rel.type;
 
-import org.eigenbase.sql.type.SqlTypeName;
+import org.apache.calcite.sql.type.SqlTypeName;
 
-/** Default implementation of {@link org.eigenbase.reltype.RelDataTypeSystem},
+/** Default implementation of
+ * {@link org.apache.calcite.rel.type.RelDataTypeSystem},
  * providing parameters from the SQL standard.
  *
  * <p>To implement other type systems, create a derived class and override

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/type/RelProtoDataType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/type/RelProtoDataType.java b/core/src/main/java/org/apache/calcite/rel/type/RelProtoDataType.java
index 2086ce3..ed518b6 100644
--- a/core/src/main/java/org/apache/calcite/rel/type/RelProtoDataType.java
+++ b/core/src/main/java/org/apache/calcite/rel/type/RelProtoDataType.java
@@ -14,15 +14,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.reltype;
+package org.apache.calcite.rel.type;
 
-import net.hydromatic.linq4j.function.Function1;
+import org.apache.calcite.linq4j.function.Function1;
 
 /**
  * Can be converted into a {@link RelDataType} given a
- * {@link org.eigenbase.reltype.RelDataTypeFactory}.
+ * {@link org.apache.calcite.rel.type.RelDataTypeFactory}.
  *
- * @see org.eigenbase.reltype.RelDataTypeImpl#proto
+ * @see org.apache.calcite.rel.type.RelDataTypeImpl#proto
  */
 public interface RelProtoDataType
     extends Function1<RelDataTypeFactory, RelDataType> {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/type/RelRecordType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/type/RelRecordType.java b/core/src/main/java/org/apache/calcite/rel/type/RelRecordType.java
index 48c937e..8d09cf4 100644
--- a/core/src/main/java/org/apache/calcite/rel/type/RelRecordType.java
+++ b/core/src/main/java/org/apache/calcite/rel/type/RelRecordType.java
@@ -14,14 +14,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.reltype;
+package org.apache.calcite.rel.type;
 
-import java.io.*;
-import java.util.List;
-
-import org.eigenbase.sql.type.*;
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.sql.type.SqlTypeName;
 
-import net.hydromatic.linq4j.Ord;
+import java.io.Serializable;
+import java.util.List;
 
 /**
  * RelRecordType represents a structured type having named fields.
@@ -89,9 +88,9 @@ public class RelRecordType extends RelDataTypeImpl implements Serializable {
   //~ Inner Classes ----------------------------------------------------------
 
   /**
-   * Skinny object which has the same information content as a {@link
-   * RelRecordType} but skips redundant stuff like digest and the immutable
-   * list.
+   * Skinny object which has the same information content as a
+   * {@link RelRecordType} but skips redundant stuff like digest and the
+   * immutable list.
    */
   private static class SerializableRelRecordType implements Serializable {
     private List<RelDataTypeField> fields;
@@ -101,8 +100,8 @@ public class RelRecordType extends RelDataTypeImpl implements Serializable {
     }
 
     /**
-     * Per {@link Serializable} API. See {@link
-     * RelRecordType#writeReplace()}.
+     * Per {@link Serializable} API. See
+     * {@link RelRecordType#writeReplace()}.
      */
     private Object readResolve() {
       return new RelRecordType(fields);

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/type/package-info.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/type/package-info.java b/core/src/main/java/org/apache/calcite/rel/type/package-info.java
index 0eaad70..7c2e251 100644
--- a/core/src/main/java/org/apache/calcite/rel/type/package-info.java
+++ b/core/src/main/java/org/apache/calcite/rel/type/package-info.java
@@ -18,6 +18,6 @@
 /**
  * Defines a type system for relational expressions.
  */
-package org.eigenbase.reltype;
+package org.apache.calcite.rel.type;
 
 // End package-info.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rex/RexAction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexAction.java b/core/src/main/java/org/apache/calcite/rex/RexAction.java
index 53ab923..497d349 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexAction.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexAction.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rex;
+package org.apache.calcite.rex;
 
 /**
  * A <code>RexAction</code> is called when a {@link RexPattern} finds a match.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 251af45..31c4eca 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexBuilder.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexBuilder.java
@@ -14,27 +14,45 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rex;
-
-import java.math.*;
-import java.util.*;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.SqlIntervalQualifier.TimeUnit;
-import org.eigenbase.sql.fun.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.util.*;
-import org.eigenbase.util14.DateTimeUtil;
-
-import net.hydromatic.avatica.ByteString;
-
-import net.hydromatic.optiq.runtime.Spaces;
-import net.hydromatic.optiq.runtime.SqlFunctions;
+package org.apache.calcite.rex;
+
+import org.apache.calcite.avatica.ByteString;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.runtime.Spaces;
+import org.apache.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.SqlAggFunction;
+import org.apache.calcite.sql.SqlIntervalQualifier;
+import org.apache.calcite.sql.SqlIntervalQualifier.TimeUnit;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlSpecialOperator;
+import org.apache.calcite.sql.SqlUtil;
+import org.apache.calcite.sql.fun.SqlCountAggFunction;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.type.ArraySqlType;
+import org.apache.calcite.sql.type.IntervalSqlType;
+import org.apache.calcite.sql.type.MapSqlType;
+import org.apache.calcite.sql.type.SqlTypeFamily;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.type.SqlTypeUtil;
+import org.apache.calcite.util.DateTimeUtil;
+import org.apache.calcite.util.NlsString;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
 
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Calendar;
+import java.util.List;
+import java.util.Map;
+
 /**
  * Factory for row expressions.
  *
@@ -151,8 +169,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 Util.newInternal("Field ordinal " + i + " is invalid for "
           + " type '" + type + "'");
     }
     return makeFieldAccessInternal(expr, fields.get(i));
@@ -199,7 +216,7 @@ public class RexBuilder {
    * Creates a call with an array of arguments.
    *
    * <p>If you already know the return type of the call, then
-   * {@link #makeCall(org.eigenbase.reltype.RelDataType, org.eigenbase.sql.SqlOperator, java.util.List)}
+   * {@link #makeCall(org.apache.calcite.rel.type.RelDataType, org.apache.calcite.sql.SqlOperator, java.util.List)}
    * is preferred.</p>
    */
   public RexNode makeCall(
@@ -752,9 +769,9 @@ public class RexBuilder {
 
   /**
    * Internal method to create a call to a literal. Code outside this package
-   * should call one of the type-specific methods such as {@link
-   * #makeDateLiteral(Calendar)}, {@link #makeLiteral(boolean)}, {@link
-   * #makeLiteral(String)}.
+   * should call one of the type-specific methods such as
+   * {@link #makeDateLiteral(Calendar)}, {@link #makeLiteral(boolean)},
+   * {@link #makeLiteral(String)}.
    *
    * @param o        Value of literal, must be appropriate for the type
    * @param type     Type of literal
@@ -1217,7 +1234,7 @@ public class RexBuilder {
   }
 
   /** Converts the type of a value to comply with
-   * {@link org.eigenbase.rex.RexLiteral#valueMatchesType}. */
+   * {@link org.apache.calcite.rex.RexLiteral#valueMatchesType}. */
   private static Object clean(Object o, RelDataType type) {
     if (o == null) {
       return null;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 accc069..0300f54 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexCall.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexCall.java
@@ -14,15 +14,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rex;
+package org.apache.calcite.rex;
 
-import java.util.*;
-
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
+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 com.google.common.collect.ImmutableList;
 
+import java.util.List;
+
 /**
  * An expression formed by a call to an operator with zero or more expressions
  * as operands.
@@ -30,9 +32,9 @@ import com.google.common.collect.ImmutableList;
  * <p>Operators may be binary, unary, functions, special syntactic constructs
  * like <code>CASE ... WHEN ... END</code>, or even internally generated
  * constructs like implicit type conversions. The syntax of the operator is
- * really irrelevant, because row-expressions (unlike {@link
- * org.eigenbase.sql.SqlNode SQL expressions}) do not directly represent a piece
- * of source code.</p>
+ * really irrelevant, because row-expressions (unlike
+ * {@link org.apache.calcite.sql.SqlNode SQL expressions})
+ * do not directly represent a piece of source code.
  *
  * <p>It's not often necessary to sub-class this class. The smarts should be in
  * the operator, rather than the call. Any extra information about the call can
@@ -109,8 +111,7 @@ public class RexCall extends RexNode {
     return type;
   }
 
-  @Override
-  public boolean isAlwaysTrue() {
+  @Override public boolean isAlwaysTrue() {
     // "c IS NOT NULL" occurs when we expand EXISTS.
     // This reduction allows us to convert it to a semi-join.
     switch (getKind()) {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rex/RexCallBinding.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexCallBinding.java b/core/src/main/java/org/apache/calcite/rex/RexCallBinding.java
index 8e5de25..fc80c19 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexCallBinding.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexCallBinding.java
@@ -14,19 +14,22 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rex;
+package org.apache.calcite.rex;
 
-import java.util.List;
-
-import org.eigenbase.reltype.*;
-import org.eigenbase.resource.Resources;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.sql.validate.*;
-import org.eigenbase.util.*;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.runtime.CalciteException;
+import org.apache.calcite.runtime.Resources;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlOperatorBinding;
+import org.apache.calcite.sql.SqlUtil;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.validate.SqlValidatorException;
 
 import com.google.common.collect.ImmutableList;
 
+import java.util.List;
+
 /**
  * <code>RexCallBinding</code> implements {@link SqlOperatorBinding} by
  * referring to an underlying collection of {@link RexNode} operands.
@@ -73,7 +76,7 @@ public class RexCallBinding extends SqlOperatorBinding {
     return operands.get(ordinal).getType();
   }
 
-  public EigenbaseException newError(
+  public CalciteException newError(
       Resources.ExInst<SqlValidatorException> e) {
     return SqlUtil.newContextException(SqlParserPos.ZERO, e);
   }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 70eca3b..77edc1f 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexChecker.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexChecker.java
@@ -14,12 +14,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rex;
+package org.apache.calcite.rex;
 
-import java.util.*;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
 
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.*;
+import java.util.List;
 
 /**
  * Visitor which checks the validity of a {@link RexNode} expression.
@@ -63,14 +64,15 @@ public class RexChecker extends RexVisitorImpl<Boolean> {
   /**
    * Creates a RexChecker with a given input row type.
    *
-   * <p>If <code>fail</code> is true, the checker will throw an {@link
-   * AssertionError} if an invalid node is found and assertions are enabled.
+   * <p>If <code>fail</code> is true, the checker will throw an
+   * {@link AssertionError} if an invalid node is found and assertions are
+   * enabled.
    *
    * <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 fail Whether to throw an {@link AssertionError} if an
+   *                     invalid node is detected
    */
   public RexChecker(final RelDataType inputRowType, boolean fail) {
     this(RelOptUtil.getFieldTypeList(inputRowType), fail);
@@ -79,14 +81,15 @@ public class RexChecker extends RexVisitorImpl<Boolean> {
   /**
    * Creates a RexChecker with a given set of input fields.
    *
-   * <p>If <code>fail</code> is true, the checker will throw an {@link
-   * AssertionError} if an invalid node is found and assertions are enabled.
+   * <p>If <code>fail</code> is true, the checker will throw an
+   * {@link AssertionError} if an invalid node is found and assertions are
+   * enabled.
    *
    * <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 fail Whether to throw an {@link AssertionError} if an
+   *                      invalid node is detected
    */
   public RexChecker(List<RelDataType> inputTypeList, boolean fail) {
     super(true);

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rex/RexCopier.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexCopier.java b/core/src/main/java/org/apache/calcite/rex/RexCopier.java
index 4a52b1a..708a59a 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexCopier.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexCopier.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rex;
+package org.apache.calcite.rex;
 
 /**
  * Shuttle which creates a deep copy of a Rex expression.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rex/RexCorrelVariable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexCorrelVariable.java b/core/src/main/java/org/apache/calcite/rex/RexCorrelVariable.java
index 6a4fc69..ea1dade 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexCorrelVariable.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexCorrelVariable.java
@@ -14,10 +14,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rex;
+package org.apache.calcite.rex;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.SqlKind;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlKind;
 
 /**
  * Reference to the current row of a correlating relational expression.
@@ -41,8 +41,7 @@ public class RexCorrelVariable extends RexVariable {
     return visitor.visitCorrelVariable(this);
   }
 
-  @Override
-  public SqlKind getKind() {
+  @Override public SqlKind getKind() {
     return SqlKind.CORREL_VARIABLE;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rex/RexDynamicParam.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexDynamicParam.java b/core/src/main/java/org/apache/calcite/rex/RexDynamicParam.java
index 5f45401..cd51cc9 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexDynamicParam.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexDynamicParam.java
@@ -14,10 +14,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rex;
+package org.apache.calcite.rex;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.SqlKind;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlKind;
 
 /**
  * Dynamic parameter reference in a row-expression.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rex/RexExecutable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexExecutable.java b/core/src/main/java/org/apache/calcite/rex/RexExecutable.java
index 8769228..64c7c99 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexExecutable.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexExecutable.java
@@ -14,7 +14,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rex;
+package org.apache.calcite.rex;
+
+import org.apache.calcite.DataContext;
+import org.apache.calcite.linq4j.function.Function1;
+import org.apache.calcite.runtime.Hook;
+import org.apache.calcite.runtime.Utilities;
+import org.apache.calcite.util.Pair;
+
+import org.codehaus.commons.compiler.CompileException;
+import org.codehaus.janino.ClassBodyEvaluator;
+import org.codehaus.janino.Scanner;
 
 import java.io.IOException;
 import java.io.Serializable;
@@ -22,18 +32,6 @@ import java.io.StringReader;
 import java.util.Arrays;
 import java.util.List;
 
-import org.eigenbase.util.Pair;
-
-import net.hydromatic.linq4j.function.Function1;
-
-import net.hydromatic.optiq.DataContext;
-import net.hydromatic.optiq.runtime.Hook;
-import net.hydromatic.optiq.runtime.Utilities;
-
-import org.codehaus.commons.compiler.CompileException;
-import org.codehaus.janino.ClassBodyEvaluator;
-import org.codehaus.janino.Scanner;
-
 /**
  * Result of compiling code generated from a {@link RexNode} expression.
  */

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rex/RexExecutorImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexExecutorImpl.java b/core/src/main/java/org/apache/calcite/rex/RexExecutorImpl.java
index da34a55..c795c23 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexExecutorImpl.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexExecutorImpl.java
@@ -14,27 +14,31 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rex;
+package org.apache.calcite.rex;
+
+import org.apache.calcite.DataContext;
+import org.apache.calcite.adapter.enumerable.RexToLixTranslator;
+import org.apache.calcite.adapter.enumerable.RexToLixTranslator.InputGetter;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
+import org.apache.calcite.linq4j.tree.BlockBuilder;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.linq4j.tree.IndexExpression;
+import org.apache.calcite.linq4j.tree.MethodCallExpression;
+import org.apache.calcite.linq4j.tree.MethodDeclaration;
+import org.apache.calcite.linq4j.tree.ParameterExpression;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.prepare.CalcitePrepareImpl;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.util.BuiltInMethod;
+
+import com.google.common.collect.ImmutableList;
 
 import java.lang.reflect.Modifier;
 import java.lang.reflect.Type;
-import java.util.*;
-
-import org.eigenbase.relopt.RelOptPlanner;
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.reltype.RelDataTypeFactory;
-
-import net.hydromatic.linq4j.expressions.*;
-
-import net.hydromatic.optiq.BuiltinMethod;
-import net.hydromatic.optiq.DataContext;
-import net.hydromatic.optiq.impl.java.JavaTypeFactory;
-import net.hydromatic.optiq.jdbc.JavaTypeFactoryImpl;
-import net.hydromatic.optiq.prepare.OptiqPrepareImpl;
-import net.hydromatic.optiq.rules.java.RexToLixTranslator;
-import net.hydromatic.optiq.rules.java.RexToLixTranslator.InputGetter;
-
-import com.google.common.collect.ImmutableList;
+import java.util.List;
 
 /**
 * Evaluates a {@link RexNode} expression.
@@ -79,10 +83,10 @@ public class RexExecutorImpl implements RelOptPlanner.Executor {
             Expressions.newArrayInit(Object[].class, expressions)));
     final MethodDeclaration methodDecl =
         Expressions.methodDecl(Modifier.PUBLIC, Object[].class,
-            BuiltinMethod.FUNCTION1_APPLY.method.getName(),
+            BuiltInMethod.FUNCTION1_APPLY.method.getName(),
             ImmutableList.of(root0_), blockBuilder.toBlock());
     String code = Expressions.toString(methodDecl);
-    if (OptiqPrepareImpl.DEBUG) {
+    if (CalcitePrepareImpl.DEBUG) {
       System.out.println(code);
     }
     return code;
@@ -124,9 +128,9 @@ public class RexExecutorImpl implements RelOptPlanner.Executor {
 
   /**
    * Implementation of
-   * {@link net.hydromatic.optiq.rules.java.RexToLixTranslator.InputGetter}
+   * {@link org.apache.calcite.adapter.enumerable.RexToLixTranslator.InputGetter}
    * that reads the values of input fields by calling
-   * <code>{@link net.hydromatic.optiq.DataContext#get}("inputRecord")</code>.
+   * <code>{@link org.apache.calcite.DataContext#get}("inputRecord")</code>.
    */
   private static class DataContextInputGetter implements InputGetter {
     private final RelDataTypeFactory typeFactory;
@@ -141,7 +145,7 @@ public class RexExecutorImpl implements RelOptPlanner.Executor {
     public Expression field(BlockBuilder list, int index, Type storageType) {
       MethodCallExpression recFromCtx = Expressions.call(
           DataContext.ROOT,
-          BuiltinMethod.DATA_CONTEXT_GET.method,
+          BuiltInMethod.DATA_CONTEXT_GET.method,
           Expressions.constant("inputRecord"));
       Expression recFromCtxCasted =
           RexToLixTranslator.convert(recFromCtx, Object[].class);

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rex/RexFieldAccess.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexFieldAccess.java b/core/src/main/java/org/apache/calcite/rex/RexFieldAccess.java
index 40869e2..09b9058 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexFieldAccess.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexFieldAccess.java
@@ -14,10 +14,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rex;
+package org.apache.calcite.rex;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.SqlKind;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.sql.SqlKind;
 
 /**
  * Access to a field of a row-expression.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rex/RexFieldCollation.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexFieldCollation.java b/core/src/main/java/org/apache/calcite/rex/RexFieldCollation.java
index 1ed80f9..46c52d8 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexFieldCollation.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexFieldCollation.java
@@ -14,16 +14,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rex;
+package org.apache.calcite.rex;
 
-import java.util.Set;
-
-import org.eigenbase.rel.RelFieldCollation;
-import org.eigenbase.sql.SqlKind;
-import org.eigenbase.util.Pair;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.util.Pair;
 
 import com.google.common.collect.ImmutableSet;
 
+import java.util.Set;
+
 /**
  * Expression combined with sort flags (DESCENDING, NULLS LAST).
  */
@@ -32,8 +32,7 @@ public class RexFieldCollation extends Pair<RexNode, ImmutableSet<SqlKind>> {
     super(left, ImmutableSet.copyOf(right));
   }
 
-  @Override
-  public String toString() {
+  @Override public String toString() {
     String s = left.toString();
     for (SqlKind operator : right) {
       switch (operator) {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rex/RexInputRef.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexInputRef.java b/core/src/main/java/org/apache/calcite/rex/RexInputRef.java
index 9b57b93..edc38da 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexInputRef.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexInputRef.java
@@ -14,13 +14,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rex;
+package org.apache.calcite.rex;
 
-import java.util.List;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.util.Pair;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.SqlKind;
-import org.eigenbase.util.Pair;
+import java.util.List;
 
 /**
  * Variable which references a field of an input relational expression.
@@ -93,8 +94,7 @@ public class RexInputRef extends RexSlot {
         field.getName());
   }
 
-  @Override
-  public SqlKind getKind() {
+  @Override public SqlKind getKind() {
     return SqlKind.INPUT_REF;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 286b9f2..d3dd1a1 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexLiteral.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexLiteral.java
@@ -14,23 +14,33 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rex;
-
-import java.io.*;
-import java.math.*;
-import java.nio.*;
-import java.nio.charset.*;
-import java.util.*;
-
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.fun.*;
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.util.*;
-import org.eigenbase.util14.*;
-
-import net.hydromatic.avatica.ByteString;
+package org.apache.calcite.rex;
+
+import org.apache.calcite.avatica.ByteString;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlCollation;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.parser.SqlParserUtil;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.util.ConversionUtil;
+import org.apache.calcite.util.DateTimeUtil;
+import org.apache.calcite.util.NlsString;
+import org.apache.calcite.util.SaffronProperties;
+import org.apache.calcite.util.Util;
+import org.apache.calcite.util.ZonelessDate;
+import org.apache.calcite.util.ZonelessDatetime;
+import org.apache.calcite.util.ZonelessTime;
+import org.apache.calcite.util.ZonelessTimestamp;
+
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.nio.charset.Charset;
+import java.util.Calendar;
+import java.util.TimeZone;
 
 /**
  * Constant value in a row-expression.
@@ -121,8 +131,8 @@ public class RexLiteral extends RexNode {
 
   /**
    * The value of this literal. Must be consistent with its type, as per
-   * {@link #valueMatchesType}. For example, you can't store an {@link
-   * Integer} value here just because you feel like it -- all numbers are
+   * {@link #valueMatchesType}. For example, you can't store an
+   * {@link Integer} value here just because you feel like it -- all numbers are
    * represented by a {@link BigDecimal}. But since this field is private, it
    * doesn't really matter how the values are stored.
    */
@@ -139,9 +149,9 @@ public class RexLiteral extends RexNode {
   /**
    * An indication of the broad type of this literal -- even if its type isn't
    * a SQL type. Sometimes this will be different than the SQL type; for
-   * example, all exact numbers, including integers have typeName {@link
-   * SqlTypeName#DECIMAL}. See {@link #valueMatchesType} for the definitive
-   * story.
+   * example, all exact numbers, including integers have typeName
+   * {@link SqlTypeName#DECIMAL}. See {@link #valueMatchesType} for the
+   * definitive story.
    */
   private final SqlTypeName typeName;
 
@@ -440,7 +450,7 @@ public class RexLiteral extends RexNode {
       if (cal == null) {
         throw Util.newInternal(
             "fromJdbcString: invalid date/time value '"
-            + literal + "'");
+                + literal + "'");
       }
       return new RexLiteral(cal, type, typeName);
     case SYMBOL:
@@ -472,8 +482,7 @@ public class RexLiteral extends RexNode {
     return type;
   }
 
-  @Override
-  public SqlKind getKind() {
+  @Override public SqlKind getKind() {
     return SqlKind.LITERAL;
   }
 
@@ -552,7 +561,7 @@ public class RexLiteral extends RexNode {
   }
 
   public int hashCode() {
-    return Util.hashV(value, type);
+    return com.google.common.base.Objects.hashCode(value, type);
   }
 
   public static int intValue(RexNode node) {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rex/RexLocalRef.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexLocalRef.java b/core/src/main/java/org/apache/calcite/rex/RexLocalRef.java
index f251bcb..9d4d1cc 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexLocalRef.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexLocalRef.java
@@ -14,13 +14,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rex;
+package org.apache.calcite.rex;
 
-import java.util.List;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.util.Util;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.SqlKind;
-import org.eigenbase.util.*;
+import java.util.List;
 
 /**
  * Local variable.
@@ -56,8 +56,7 @@ public class RexLocalRef extends RexSlot {
 
   //~ Methods ----------------------------------------------------------------
 
-  @Override
-  public SqlKind getKind() {
+  @Override public SqlKind getKind() {
     return SqlKind.LOCAL_REF;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rex/RexMultisetUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexMultisetUtil.java b/core/src/main/java/org/apache/calcite/rex/RexMultisetUtil.java
index 63f4449..34f4b5f 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexMultisetUtil.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexMultisetUtil.java
@@ -14,16 +14,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rex;
+package org.apache.calcite.rex;
 
-import java.util.*;
-
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.fun.*;
-import org.eigenbase.sql.type.*;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.type.SqlTypeName;
 
 import com.google.common.collect.ImmutableSet;
 
+import java.util.List;
+import java.util.Set;
+
 /**
  * Utility class for various methods related to multisets.
  */

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rex/RexNode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexNode.java b/core/src/main/java/org/apache/calcite/rex/RexNode.java
index 3d39216..897c06c 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexNode.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexNode.java
@@ -14,19 +14,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rex;
+package org.apache.calcite.rex;
 
-import java.util.Collection;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlKind;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.SqlKind;
+import java.util.Collection;
 
 /**
  * Row expression.
  *
- * <p>Every row-expression has a type. (Compare with {@link
- * org.eigenbase.sql.SqlNode}, which is created before validation, and therefore
- * types may not be available.)</p>
+ * <p>Every row-expression has a type.
+ * (Compare with {@link org.apache.calcite.sql.SqlNode}, which is created before
+ * validation, and therefore types may not be available.)
  *
  * <p>Some common row-expressions are: {@link RexLiteral} (constant value),
  * {@link RexVariable} (variable), {@link RexCall} (call to operator with
@@ -83,8 +83,8 @@ public abstract class RexNode {
   }
 
   /**
-   * Accepts a visitor, dispatching to the right overloaded {@link
-   * RexVisitor#visitInputRef visitXxx} method.
+   * Accepts a visitor, dispatching to the right overloaded
+   * {@link RexVisitor#visitInputRef visitXxx} method.
    *
    * <p>Also see {@link RexProgram#apply(RexVisitor, java.util.List, RexNode)},
    * which applies a visitor to several expressions simultaneously.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rex/RexOver.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexOver.java b/core/src/main/java/org/apache/calcite/rex/RexOver.java
index a45c218..babfacc 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexOver.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexOver.java
@@ -14,13 +14,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rex;
+package org.apache.calcite.rex;
 
-import java.util.List;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlAggFunction;
+import org.apache.calcite.sql.SqlWindow;
+import org.apache.calcite.util.ControlFlowException;
+import org.apache.calcite.util.Util;
 
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.util.*;
+import java.util.List;
 
 /**
  * Call to an aggregate function over a window.
@@ -41,7 +43,7 @@ public class RexOver extends RexCall {
    *
    * <ul>
    * <li>type = Integer,
-   * <li>op = {@link org.eigenbase.sql.fun.SqlStdOperatorTable#SUM},
+   * <li>op = {@link org.apache.calcite.sql.fun.SqlStdOperatorTable#SUM},
    * <li>operands = { {@link RexFieldAccess}("x") }
    * <li>window = {@link SqlWindow}(ROWS 3 PRECEDING)
    * </ul>
@@ -126,13 +128,13 @@ public class RexOver extends RexCall {
     }
   }
 
-  @Override
-  public RexCall clone(RelDataType type, List<RexNode> operands) {
+  @Override public RexCall clone(RelDataType type, List<RexNode> operands) {
     throw new UnsupportedOperationException();
   }
 
   //~ Inner Classes ----------------------------------------------------------
 
+  /** Exception thrown when an OVER is found. */
   private static class OverFound extends ControlFlowException {
     public static final OverFound INSTANCE = new OverFound();
   }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rex/RexPattern.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexPattern.java b/core/src/main/java/org/apache/calcite/rex/RexPattern.java
index 8b7348f..5db7a26 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexPattern.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexPattern.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rex;
+package org.apache.calcite.rex;
 
 /**
  * A <code>RexPattern</code> represents an expression with holes in it. The

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rex/RexPermutationShuttle.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexPermutationShuttle.java b/core/src/main/java/org/apache/calcite/rex/RexPermutationShuttle.java
index f53424b..2af15ea 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexPermutationShuttle.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexPermutationShuttle.java
@@ -14,9 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rex;
+package org.apache.calcite.rex;
 
-import org.eigenbase.util.*;
+import org.apache.calcite.util.Permutation;
 
 /**
  * Visitor which replaces {@link RexLocalRef} objects after the expressions in a

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rex/RexPermuteInputsShuttle.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexPermuteInputsShuttle.java b/core/src/main/java/org/apache/calcite/rex/RexPermuteInputsShuttle.java
index f9d7b1e..1750477 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexPermuteInputsShuttle.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexPermuteInputsShuttle.java
@@ -14,21 +14,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rex;
+package org.apache.calcite.rex;
 
-import java.util.List;
-
-import org.eigenbase.rel.RelNode;
-import org.eigenbase.reltype.RelDataTypeField;
-import org.eigenbase.util.mapping.Mappings;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.util.mapping.Mappings;
 
 import com.google.common.collect.ImmutableList;
 
+import java.util.List;
+
 /**
  * Shuttle which applies a permutation to its input fields.
  *
  * @see RexPermutationShuttle
- * @see RexUtil#apply(org.eigenbase.util.mapping.Mappings.TargetMapping, RexNode)
+ * @see RexUtil#apply(org.apache.calcite.util.mapping.Mappings.TargetMapping, RexNode)
  */
 public class RexPermuteInputsShuttle extends RexShuttle {
   //~ Instance fields --------------------------------------------------------
@@ -43,7 +43,7 @@ public class RexPermuteInputsShuttle extends RexShuttle {
    *
    * <p>The mapping provides at most one target for every source. If a source
    * has no targets and is referenced in the expression,
-   * {@link org.eigenbase.util.mapping.Mappings.TargetMapping#getTarget(int)}
+   * {@link org.apache.calcite.util.mapping.Mappings.TargetMapping#getTarget(int)}
    * will give an error. Otherwise the mapping gives a unique target.
    *
    * @param mapping Mapping
@@ -80,8 +80,7 @@ public class RexPermuteInputsShuttle extends RexShuttle {
     return fields.build();
   }
 
-  @Override
-  public RexNode visitInputRef(RexInputRef local) {
+  @Override public RexNode visitInputRef(RexInputRef local) {
     final int index = local.getIndex();
     int target = mapping.getTarget(index);
     return new RexInputRef(
@@ -89,8 +88,7 @@ public class RexPermuteInputsShuttle extends RexShuttle {
         local.getType());
   }
 
-  @Override
-  public RexNode visitCall(RexCall call) {
+  @Override public RexNode visitCall(RexCall call) {
     if (call.getOperator() == RexBuilder.GET_OPERATOR) {
       final String name =
           (String) ((RexLiteral) call.getOperands().get(1)).getValue2();

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 aafe361..03f7540 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexProgram.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexProgram.java
@@ -14,23 +14,34 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rex;
-
-import java.io.PrintWriter;
-import java.io.StringWriter;
-import java.util.*;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.fun.SqlStdOperatorTable;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.util.*;
+package org.apache.calcite.rex;
+
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollationImpl;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.RelWriter;
+import org.apache.calcite.rel.externalize.RelWriterImpl;
+import org.apache.calcite.rel.type.RelDataType;
+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.Pair;
+import org.apache.calcite.util.Permutation;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.util.AbstractList;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+
 /**
  * A collection of expressions which read inputs, compute output expressions,
  * and optionally use a condition to filter rows.
@@ -251,11 +262,10 @@ public class RexProgram {
       SqlExplainLevel level) {
     final List<RelDataTypeField> inFields = inputRowType.getFieldList();
     final List<RelDataTypeField> outFields = outputRowType.getFieldList();
-    assert outFields.size() == projects.size() : "outFields.length="
-        + outFields.size()
+    assert outFields.size() == projects.size()
+        : "outFields.length=" + outFields.size()
         + ", projects.length=" + projects.size();
-    pw.item(
-        prefix + "expr#0"
+    pw.item(prefix + "expr#0"
             + ((inFields.size() > 1) ? (".." + (inFields.size() - 1)) : ""),
         "{inputs}");
     for (int i = inFields.size(); i < exprs.size(); i++) {
@@ -448,8 +458,7 @@ public class RexProgram {
                 return exprs.get(index).getType();
               }
 
-              @Override
-              public int size() {
+              @Override public int size() {
                 return exprs.size();
               }
               // CHECKSTYLE: IGNORE 1
@@ -485,7 +494,7 @@ public class RexProgram {
   /**
    * Returns whether an expression always evaluates to null.
    *
-   * <p>Like {@link RexUtil#isNull(RexNode)}, null literals are null, and                                                         |
+   * <p>Like {@link RexUtil#isNull(RexNode)}, null literals are null, and
    * casts of null literals are null. But this method also regards references
    * to null expressions as null.</p>
    *
@@ -601,7 +610,8 @@ public class RexProgram {
   public boolean projectsIdentity(final boolean fail) {
     final int fieldCount = inputRowType.getFieldCount();
     if (projects.size() < fieldCount) {
-      assert !fail : "program '" + toString()
+      assert !fail
+          : "program '" + toString()
           + "' does not project identity for input row type '"
           + inputRowType + "'";
       return false;
@@ -609,7 +619,8 @@ public class RexProgram {
     for (int i = 0; i < fieldCount; i++) {
       RexLocalRef project = projects.get(i);
       if (project.index != i) {
-        assert !fail : "program " + toString()
+        assert !fail
+            : "program " + toString()
             + "' does not project identity for input row type '"
             + inputRowType + "', field #" + i;
         return false;
@@ -849,8 +860,8 @@ public class RexProgram {
   }
 
   /**
-   * A RexShuttle used in the implementation of {@link
-   * RexProgram#expandLocalRef}.
+   * A RexShuttle used in the implementation of
+   * {@link RexProgram#expandLocalRef}.
    */
   private class ExpansionShuttle extends RexShuttle {
     public RexNode visitLocalRef(RexLocalRef localRef) {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 6567531..fa4233e 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexProgramBuilder.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexProgramBuilder.java
@@ -14,14 +14,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rex;
+package org.apache.calcite.rex;
 
-import java.util.*;
+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.Pair;
+import org.apache.calcite.util.Util;
 
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.fun.*;
-import org.eigenbase.util.*;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
 
 /**
  * Workspace for constructing a {@link RexProgram}.
@@ -142,22 +147,19 @@ public class RexProgramBuilder {
             if (index < fields.size()) {
               final RelDataTypeField inputField = fields.get(index);
               if (input.getType() != inputField.getType()) {
-                throw Util.newInternal(
-                    "in expression " + expr
+                throw Util.newInternal("in expression " + expr
                     + ", field reference " + input
                     + " has inconsistent type");
               }
             } else {
               if (index >= fieldOrdinal) {
-                throw Util.newInternal(
-                    "in expression " + expr
+                throw Util.newInternal("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
+                throw Util.newInternal("in expression " + expr
                     + ", field reference " + input
                     + " has inconsistent type");
               }
@@ -838,6 +840,8 @@ public class RexProgramBuilder {
 
   //~ Inner Classes ----------------------------------------------------------
 
+  /** Shuttle that visits a tree of {@link RexNode} and registers them
+   * in a program. */
   private abstract class RegisterShuttle extends RexShuttle {
     public RexNode visitCall(RexCall call) {
       final RexNode expr = super.visitCall(call);
@@ -888,8 +892,8 @@ public class RexProgramBuilder {
         // The expression should already be valid. Check that its
         // index is within bounds.
         if ((index < 0) || (index >= inputRowType.getFieldCount())) {
-          assert false : "RexInputRef index " + index
-              + " out of range 0.."
+          assert false
+              : "RexInputRef index " + index + " out of range 0.."
               + (inputRowType.getFieldCount() - 1);
         }
 
@@ -913,8 +917,8 @@ public class RexProgramBuilder {
         // The expression should already be valid.
         final int index = local.getIndex();
         assert index >= 0 : index;
-        assert index < exprList.size() : "index=" + index
-            + ", exprList=" + exprList;
+        assert index < exprList.size()
+            : "index=" + index + ", exprList=" + exprList;
         assert RelOptUtil.eq(
             "expr type",
             exprList.get(index).getType(),
@@ -930,8 +934,7 @@ public class RexProgramBuilder {
         if (expr instanceof RexLocalRef) {
           local = (RexLocalRef) expr;
           if (local.index >= index) {
-            throw Util.newInternal(
-                "expr " + local
+            throw Util.newInternal("expr " + local
                 + " references later expr " + local.index);
           }
         } else {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rex/RexRangeRef.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexRangeRef.java b/core/src/main/java/org/apache/calcite/rex/RexRangeRef.java
index 9ab0f79..bf2104c 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexRangeRef.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexRangeRef.java
@@ -14,17 +14,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rex;
+package org.apache.calcite.rex;
 
-import org.eigenbase.reltype.*;
+import org.apache.calcite.rel.type.RelDataType;
 
 /**
  * Reference to a range of columns.
  *
- * <p>This construct is used only during the process of translating a {@link
- * org.eigenbase.sql.SqlNode SQL} tree to a {@link org.eigenbase.rel.RelNode
- * rel}/{@link RexNode rex} tree. <em>Regular {@link RexNode rex} trees do not
- * contain this construct.</em></p>
+ * <p>This construct is used only during the process of translating a
+ * {@link org.apache.calcite.sql.SqlNode SQL} tree to a
+ * {@link org.apache.calcite.rel.RelNode rel}/{@link RexNode rex}
+ * tree. <em>Regular {@link RexNode rex} trees do not contain this
+ * construct.</em></p>
  *
  * <p>While translating a join of EMP(EMPNO, ENAME, DEPTNO) to DEPT(DEPTNO2,
  * DNAME) we create <code>RexRangeRef(DeptType,3)</code> to represent the pair
@@ -33,7 +34,7 @@ import org.eigenbase.reltype.*;
  *
  * <p>Suppose we later create a reference to the DNAME field of this
  * RexRangeRef; it will return a <code>{@link RexInputRef}(5,Integer)</code>,
- * and the {@link org.eigenbase.rex.RexRangeRef} will disappear.</p>
+ * and the {@link org.apache.calcite.rex.RexRangeRef} will disappear.</p>
  */
 public class RexRangeRef extends RexNode {
   //~ Instance fields --------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rex/RexShuttle.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexShuttle.java b/core/src/main/java/org/apache/calcite/rex/RexShuttle.java
index 9649b62..bd3ce46 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexShuttle.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexShuttle.java
@@ -14,19 +14,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rex;
-
-import java.util.*;
+package org.apache.calcite.rex;
 
 import com.google.common.collect.ImmutableList;
 
+import java.util.ArrayList;
+import java.util.List;
+
 /**
  * Passes over a row-expression, calling a handler method for each node,
  * appropriate to the type of the node.
  *
- * <p>Like {@link RexVisitor}, this is an instance of the {@link
- * org.eigenbase.util.Glossary#VISITOR_PATTERN Visitor Pattern}. Use <code>
- * RexShuttle</code> if you would like your methods to return a value.</p>
+ * <p>Like {@link RexVisitor}, this is an instance of the
+ * {@link org.apache.calcite.util.Glossary#VISITOR_PATTERN Visitor Pattern}. Use
+ * <code> RexShuttle</code> if you would like your methods to return a
+ * value.</p>
  */
 public class RexShuttle implements RexVisitor<RexNode> {
   //~ Methods ----------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rex/RexSlot.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexSlot.java b/core/src/main/java/org/apache/calcite/rex/RexSlot.java
index 11c0e62..5421f12 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexSlot.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexSlot.java
@@ -14,13 +14,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rex;
+package org.apache.calcite.rex;
+
+import org.apache.calcite.rel.type.RelDataType;
 
 import java.util.AbstractList;
 import java.util.concurrent.CopyOnWriteArrayList;
 
-import org.eigenbase.reltype.*;
-
 /**
  * Abstract base class for {@link RexInputRef} and {@link RexLocalRef}.
  */
@@ -81,8 +81,7 @@ public abstract class RexSlot extends RexVariable {
       };
     }
 
-    @Override
-    public String get(int index) {
+    @Override public String get(int index) {
       for (;;) {
         try {
           return super.get(index);

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rex/RexSqlConvertlet.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexSqlConvertlet.java b/core/src/main/java/org/apache/calcite/rex/RexSqlConvertlet.java
index 7107c00..2aca4ea 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexSqlConvertlet.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexSqlConvertlet.java
@@ -14,9 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rex;
+package org.apache.calcite.rex;
 
-import org.eigenbase.sql.*;
+import org.apache.calcite.sql.SqlNode;
 
 /**
  * Converts a {@link RexNode} expression into a {@link SqlNode} expression.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rex/RexSqlConvertletTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexSqlConvertletTable.java b/core/src/main/java/org/apache/calcite/rex/RexSqlConvertletTable.java
index dbec59c..8f00b4f 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexSqlConvertletTable.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexSqlConvertletTable.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rex;
+package org.apache.calcite.rex;
 
 /**
  * Collection of {@link RexSqlConvertlet}s.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rex/RexSqlReflectiveConvertletTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexSqlReflectiveConvertletTable.java b/core/src/main/java/org/apache/calcite/rex/RexSqlReflectiveConvertletTable.java
index 4619b98..596ebd5 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexSqlReflectiveConvertletTable.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexSqlReflectiveConvertletTable.java
@@ -14,11 +14,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rex;
+package org.apache.calcite.rex;
 
-import java.util.*;
+import org.apache.calcite.sql.SqlOperator;
 
-import org.eigenbase.sql.*;
+import java.util.HashMap;
+import java.util.Map;
 
 /**
  * Implementation of {@link RexSqlConvertletTable}.
@@ -73,8 +74,8 @@ public class RexSqlReflectiveConvertletTable implements RexSqlConvertletTable {
   /**
    * Registers a convertlet for a given operator instance
    *
-   * @param op         Operator instance, say {@link
-   *                   org.eigenbase.sql.fun.SqlStdOperatorTable#MINUS}
+   * @param op         Operator instance, say
+   * {@link org.apache.calcite.sql.fun.SqlStdOperatorTable#MINUS}
    * @param convertlet Convertlet
    */
   protected void registerOp(SqlOperator op, RexSqlConvertlet convertlet) {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rex/RexSqlStandardConvertletTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexSqlStandardConvertletTable.java b/core/src/main/java/org/apache/calcite/rex/RexSqlStandardConvertletTable.java
index 704e933..0745995 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexSqlStandardConvertletTable.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexSqlStandardConvertletTable.java
@@ -14,14 +14,22 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rex;
+package org.apache.calcite.rex;
 
-import java.util.*;
+import org.apache.calcite.sql.SqlBasicCall;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlDataTypeSpec;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlNodeList;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.fun.SqlCaseOperator;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.SqlTypeUtil;
 
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.fun.*;
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.sql.type.*;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
 
 /**
  * Standard implementation of {@link RexSqlConvertletTable}.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rex/RexToSqlNodeConverter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexToSqlNodeConverter.java b/core/src/main/java/org/apache/calcite/rex/RexToSqlNodeConverter.java
index 7f095b7..3e2a966 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexToSqlNodeConverter.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexToSqlNodeConverter.java
@@ -14,9 +14,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rex;
+package org.apache.calcite.rex;
 
-import org.eigenbase.sql.*;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlNode;
 
 /**
  * Converts expressions from {@link RexNode} to {@link SqlNode}.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rex/RexToSqlNodeConverterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexToSqlNodeConverterImpl.java b/core/src/main/java/org/apache/calcite/rex/RexToSqlNodeConverterImpl.java
index 65d91df..16b4203 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexToSqlNodeConverterImpl.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexToSqlNodeConverterImpl.java
@@ -14,14 +14,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rex;
+package org.apache.calcite.rex;
 
-import java.util.*;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.SqlTypeFamily;
+import org.apache.calcite.util.NlsString;
 
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.util.*;
+import java.util.Calendar;
 
 /**
  * Standard implementation of {@link RexToSqlNodeConverter}.


[16/58] [abbrv] [partial] incubator-calcite git commit: [CALCITE-306] Standardize code style for "import package.*; "

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/test/DiffRepository.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/DiffRepository.java b/core/src/test/java/org/apache/calcite/test/DiffRepository.java
index f131833..ae06e94 100644
--- a/core/src/test/java/org/apache/calcite/test/DiffRepository.java
+++ b/core/src/test/java/org/apache/calcite/test/DiffRepository.java
@@ -14,20 +14,33 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.test;
-
-import java.io.*;
+package org.apache.calcite.test;
+
+import org.apache.calcite.util.Util;
+import org.apache.calcite.util.XmlOutput;
+
+import org.junit.Assert;
+import org.junit.ComparisonFailure;
+import org.w3c.dom.CDATASection;
+import org.w3c.dom.Comment;
+import org.w3c.dom.Document;
+import org.w3c.dom.Element;
+import org.w3c.dom.NamedNodeMap;
+import org.w3c.dom.Node;
+import org.w3c.dom.NodeList;
+import org.w3c.dom.Text;
+import org.xml.sax.SAXException;
+
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.Writer;
 import java.net.URL;
-import java.util.*;
-import javax.xml.parsers.*;
-
-import org.eigenbase.util.*;
-
-import org.junit.*;
-
-import org.w3c.dom.*;
-
-import org.xml.sax.*;
+import java.util.HashMap;
+import java.util.Map;
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.DocumentBuilderFactory;
+import javax.xml.parsers.ParserConfigurationException;
 
 /**
  * A collection of resources used by tests.
@@ -195,8 +208,7 @@ public class DiffRepository {
       }
       this.root = doc.getDocumentElement();
       if (!root.getNodeName().equals(ROOT_TAG)) {
-        throw new RuntimeException(
-            "expected root element of type '" + ROOT_TAG
+        throw new RuntimeException("expected root element of type '" + ROOT_TAG
             + "', but found '" + root.getNodeName() + "'");
       }
     } catch (ParserConfigurationException e) {
@@ -401,10 +413,8 @@ public class DiffRepository {
     String expected2 = expand(tag, expected);
     if (expected2 == null) {
       update(testCaseName, expected, actual);
-      throw new AssertionError(
-          "reference file does not contain resource '" + expected
-          + "' for testcase '" + testCaseName
-          + "'");
+      throw new AssertionError("reference file does not contain resource '"
+          + expected + "' for testcase '" + testCaseName + "'");
     } else {
       try {
         // TODO jvs 25-Apr-2006:  reuse bulk of
@@ -600,8 +610,7 @@ public class DiffRepository {
       break;
 
     default:
-      throw new RuntimeException(
-          "unexpected node type: " + node.getNodeType()
+      throw new RuntimeException("unexpected node type: " + node.getNodeType()
           + " (" + node + ")");
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/test/DiffTestCase.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/DiffTestCase.java b/core/src/test/java/org/apache/calcite/test/DiffTestCase.java
index 8b5e069..7e3adee 100644
--- a/core/src/test/java/org/apache/calcite/test/DiffTestCase.java
+++ b/core/src/test/java/org/apache/calcite/test/DiffTestCase.java
@@ -14,20 +14,29 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.test;
-
-import java.io.*;
-
-import java.util.*;
-import java.util.regex.*;
-
-import org.eigenbase.util.*;
-
-import org.incava.util.diff.*;
-
-import org.junit.*;
-
-import static org.junit.Assert.*;
+package org.apache.calcite.test;
+
+import org.apache.calcite.util.ReflectUtil;
+
+import org.incava.util.diff.Diff;
+import org.incava.util.diff.Difference;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.FileReader;
+import java.io.IOException;
+import java.io.LineNumberReader;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.io.StringWriter;
+import java.io.Writer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
 
 /**
  * DiffTestCase is an abstract base for JUnit tests which produce multi-line

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/test/ExceptionMessageTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/ExceptionMessageTest.java b/core/src/test/java/org/apache/calcite/test/ExceptionMessageTest.java
index ec2f757..6b42963 100644
--- a/core/src/test/java/org/apache/calcite/test/ExceptionMessageTest.java
+++ b/core/src/test/java/org/apache/calcite/test/ExceptionMessageTest.java
@@ -14,11 +14,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.test;
+package org.apache.calcite.test;
 
-import net.hydromatic.optiq.SchemaPlus;
-import net.hydromatic.optiq.impl.java.ReflectiveSchema;
-import net.hydromatic.optiq.jdbc.OptiqConnection;
+import org.apache.calcite.adapter.java.ReflectiveSchema;
+import org.apache.calcite.jdbc.CalciteConnection;
+import org.apache.calcite.schema.SchemaPlus;
 
 import org.junit.Before;
 import org.junit.Test;
@@ -29,8 +29,10 @@ import java.sql.SQLException;
 import java.sql.Statement;
 import java.util.Iterator;
 
-import static org.hamcrest.CoreMatchers.*;
-import static org.junit.Assert.*;
+import static org.hamcrest.CoreMatchers.containsString;
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
 
 /**
  * Test cases to check that necessary information from underlying exceptions
@@ -72,11 +74,12 @@ public class ExceptionMessageTest {
   @Before
   public void setUp() throws SQLException {
     Connection connection = DriverManager.getConnection("jdbc:calcite:");
-    OptiqConnection optiqConnection = connection.unwrap(OptiqConnection.class);
-    SchemaPlus rootSchema = optiqConnection.getRootSchema();
+    CalciteConnection calciteConnection =
+        connection.unwrap(CalciteConnection.class);
+    SchemaPlus rootSchema = calciteConnection.getRootSchema();
     rootSchema.add("test", new ReflectiveSchema(new TestSchema()));
-    optiqConnection.setSchema("test");
-    this.conn = optiqConnection;
+    calciteConnection.setSchema("test");
+    this.conn = calciteConnection;
   }
 
   private void runQuery(String sql) throws SQLException {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/test/FilteratorTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/FilteratorTest.java b/core/src/test/java/org/apache/calcite/test/FilteratorTest.java
index d2f1dfb..b0d8d94 100644
--- a/core/src/test/java/org/apache/calcite/test/FilteratorTest.java
+++ b/core/src/test/java/org/apache/calcite/test/FilteratorTest.java
@@ -14,15 +14,23 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.test;
+package org.apache.calcite.test;
 
-import java.util.*;
-
-import org.eigenbase.util.*;
+import org.apache.calcite.util.Filterator;
+import org.apache.calcite.util.Util;
 
 import org.junit.Test;
 
-import static org.junit.Assert.*;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
 
 /**
  * Unit test for {@link Filterator}.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/test/FoodmartTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/FoodmartTest.java b/core/src/test/java/org/apache/calcite/test/FoodmartTest.java
index c671809..a8999f1 100644
--- a/core/src/test/java/org/apache/calcite/test/FoodmartTest.java
+++ b/core/src/test/java/org/apache/calcite/test/FoodmartTest.java
@@ -14,27 +14,29 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.test;
+package org.apache.calcite.test;
 
-import net.hydromatic.linq4j.expressions.Primitive;
-
-import org.eigenbase.util.IntegerIntervalSet;
+import org.apache.calcite.linq4j.tree.Primitive;
+import org.apache.calcite.util.IntegerIntervalSet;
 
 import com.fasterxml.jackson.core.JsonParser;
 import com.fasterxml.jackson.databind.ObjectMapper;
-
 import com.google.common.collect.ImmutableList;
 
+import mondrian.test.data.FoodMartQuery;
+
 import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 
-import java.io.*;
+import java.io.IOException;
+import java.io.InputStream;
 import java.lang.ref.SoftReference;
-import java.util.*;
-
-import mondrian.test.data.FoodMartQuery;
+import java.util.ArrayList;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
 
 /**
  * Test case that runs the FoodMart reference queries.
@@ -106,7 +108,7 @@ public class FoodmartTest {
   @Parameterized.Parameters(name = "{index}: foodmart({0})={1}")
   public static List<Object[]> getSqls() throws IOException {
     String idList = System.getProperty("calcite.ids");
-    if (!OptiqAssert.ENABLE_SLOW && idList == null) {
+    if (!CalciteAssert.ENABLE_SLOW && idList == null) {
       // Avoid loading the query set in a regular test suite run. It burns too
       // much memory.
       return ImmutableList.of();
@@ -130,7 +132,7 @@ public class FoodmartTest {
       }
     } else {
       for (FoodmartQuery query1 : set.queries.values()) {
-        if (!OptiqAssert.ENABLE_SLOW && query1.id != 2) {
+        if (!CalciteAssert.ENABLE_SLOW && query1.id != 2) {
           // If slow queries are not enabled, only run query #2.
           continue;
         }
@@ -151,8 +153,8 @@ public class FoodmartTest {
   @Test(timeout = 60000)
   public void test() {
     try {
-      OptiqAssert.that()
-          .with(OptiqAssert.Config.FOODMART_CLONE)
+      CalciteAssert.that()
+          .with(CalciteAssert.Config.FOODMART_CLONE)
           .pooled()
           .query(query.sql)
           .runs();
@@ -166,8 +168,8 @@ public class FoodmartTest {
   @Ignore
   public void testWithLattice() {
     try {
-      OptiqAssert.that()
-          .with(OptiqAssert.Config.JDBC_FOODMART_WITH_LATTICE)
+      CalciteAssert.that()
+          .with(CalciteAssert.Config.JDBC_FOODMART_WITH_LATTICE)
           .pooled()
           .withSchema("foodmart")
           .query(query.sql)
@@ -179,6 +181,7 @@ public class FoodmartTest {
     }
   }
 
+  /** Set of queries against the FoodMart database. */
   public static class FoodMartQuerySet {
     private static SoftReference<FoodMartQuerySet> ref;
 
@@ -214,12 +217,12 @@ public class FoodmartTest {
     }
   }
 
-  // JSON class
+  /** JSON root element. */
   public static class FoodmartRoot {
     public final List<FoodmartQuery> queries = new ArrayList<FoodmartQuery>();
   }
 
-  // JSON class
+  /** JSON query element. */
   public static class FoodmartQuery {
     public int id;
     public String sql;
@@ -227,7 +230,7 @@ public class FoodmartTest {
     public final List<List> rows = new ArrayList<List>();
   }
 
-  // JSON class
+  /** JSON column element. */
   public static class FoodmartColumn {
     public String name;
     public String type;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/test/HepPlannerTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/HepPlannerTest.java b/core/src/test/java/org/apache/calcite/test/HepPlannerTest.java
index a408950..ac8f67a 100644
--- a/core/src/test/java/org/apache/calcite/test/HepPlannerTest.java
+++ b/core/src/test/java/org/apache/calcite/test/HepPlannerTest.java
@@ -14,18 +14,27 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.test;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.rel.rules.*;
-import org.eigenbase.relopt.hep.*;
+package org.apache.calcite.test;
+
+import org.apache.calcite.plan.hep.HepMatchOrder;
+import org.apache.calcite.plan.hep.HepPlanner;
+import org.apache.calcite.plan.hep.HepProgram;
+import org.apache.calcite.plan.hep.HepProgramBuilder;
+import org.apache.calcite.rel.logical.LogicalIntersect;
+import org.apache.calcite.rel.logical.LogicalUnion;
+import org.apache.calcite.rel.rules.CalcMergeRule;
+import org.apache.calcite.rel.rules.CoerceInputsRule;
+import org.apache.calcite.rel.rules.FilterToCalcRule;
+import org.apache.calcite.rel.rules.ProjectRemoveRule;
+import org.apache.calcite.rel.rules.ProjectToCalcRule;
+import org.apache.calcite.rel.rules.UnionToDistinctRule;
 
 import org.junit.Test;
 
 /**
- * HepPlannerTest is a unit test for {@link HepPlanner}. See {@link
- * RelOptRulesTest} for an explanation of how to add tests; the tests in this
- * class are targeted at exercising the planner, and use specific rules for
+ * HepPlannerTest is a unit test for {@link HepPlanner}. See
+ * {@link RelOptRulesTest} for an explanation of how to add tests; the tests in
+ * this class are targeted at exercising the planner, and use specific rules for
  * convenience only, whereas the tests in that class are targeted at exercising
  * specific rules, and use the planner for convenience only. Hence the split.
  */
@@ -52,13 +61,12 @@ public class HepPlannerTest extends RelOptTestBase {
         new HepPlanner(
             programBuilder.build());
 
-    planner.addRule(new CoerceInputsRule(UnionRel.class, false));
-    planner.addRule(new CoerceInputsRule(IntersectRel.class, false));
+    planner.addRule(new CoerceInputsRule(LogicalUnion.class, false));
+    planner.addRule(new CoerceInputsRule(LogicalIntersect.class, false));
 
-    checkPlanning(
-        planner,
+    checkPlanning(planner,
         "(select name from dept union select ename from emp)"
-        + " intersect (select fname from customer.contact)");
+            + " intersect (select fname from customer.contact)");
   }
 
   @Test public void testRuleDescription() throws Exception {
@@ -116,15 +124,15 @@ public class HepPlannerTest extends RelOptTestBase {
   @Test public void testReplaceCommonSubexpression() throws Exception {
     // Note that here it may look like the rule is firing
     // twice, but actually it's only firing once on the
-    // common subexpression.  The purpose of this test
+    // common sub-expression.  The purpose of this test
     // is to make sure the planner can deal with
-    // rewriting something used as a common subexpression
+    // rewriting something used as a common sub-expression
     // twice by the same parent (the join in this case).
 
     checkPlanning(
-        RemoveTrivialProjectRule.INSTANCE,
+        ProjectRemoveRule.INSTANCE,
         "select d1.deptno from (select * from dept) d1,"
-        + " (select * from dept) d2");
+            + " (select * from dept) d2");
   }
 
   @Test public void testSubprogram() throws Exception {
@@ -136,7 +144,7 @@ public class HepPlannerTest extends RelOptTestBase {
     subprogramBuilder.addMatchOrder(HepMatchOrder.TOP_DOWN);
     subprogramBuilder.addMatchLimit(1);
     subprogramBuilder.addRuleInstance(ProjectToCalcRule.INSTANCE);
-    subprogramBuilder.addRuleInstance(MergeCalcRule.INSTANCE);
+    subprogramBuilder.addRuleInstance(CalcMergeRule.INSTANCE);
 
     HepProgramBuilder programBuilder = HepProgram.builder();
     programBuilder.addSubprogram(subprogramBuilder.build());
@@ -152,7 +160,7 @@ public class HepPlannerTest extends RelOptTestBase {
     // that order doesn't matter within the group.
     HepProgramBuilder programBuilder = HepProgram.builder();
     programBuilder.addGroupBegin();
-    programBuilder.addRuleInstance(MergeCalcRule.INSTANCE);
+    programBuilder.addRuleInstance(CalcMergeRule.INSTANCE);
     programBuilder.addRuleInstance(ProjectToCalcRule.INSTANCE);
     programBuilder.addRuleInstance(FilterToCalcRule.INSTANCE);
     programBuilder.addGroupEnd();

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/test/InterpreterTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/InterpreterTest.java b/core/src/test/java/org/apache/calcite/test/InterpreterTest.java
index 6364b2e..0cc0fcf 100644
--- a/core/src/test/java/org/apache/calcite/test/InterpreterTest.java
+++ b/core/src/test/java/org/apache/calcite/test/InterpreterTest.java
@@ -14,21 +14,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.test;
-
-import net.hydromatic.linq4j.QueryProvider;
-
-import net.hydromatic.optiq.DataContext;
-import net.hydromatic.optiq.SchemaPlus;
-import net.hydromatic.optiq.config.Lex;
-import net.hydromatic.optiq.impl.interpreter.Interpreter;
-import net.hydromatic.optiq.impl.java.JavaTypeFactory;
-import net.hydromatic.optiq.tools.FrameworkConfig;
-import net.hydromatic.optiq.tools.Frameworks;
-import net.hydromatic.optiq.tools.Planner;
-
-import org.eigenbase.rel.RelNode;
-import org.eigenbase.sql.SqlNode;
+package org.apache.calcite.test;
+
+import org.apache.calcite.DataContext;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.config.Lex;
+import org.apache.calcite.interpreter.Interpreter;
+import org.apache.calcite.linq4j.QueryProvider;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.tools.FrameworkConfig;
+import org.apache.calcite.tools.Frameworks;
+import org.apache.calcite.tools.Planner;
 
 import com.google.common.collect.Lists;
 
@@ -43,7 +41,7 @@ import static org.hamcrest.CoreMatchers.equalTo;
 import static org.junit.Assert.assertThat;
 
 /**
- * Unit tests for {@link net.hydromatic.optiq.impl.interpreter.Interpreter}.
+ * Unit tests for {@link org.apache.calcite.interpreter.Interpreter}.
  */
 public class InterpreterTest {
   private SchemaPlus rootSchema;
@@ -80,7 +78,7 @@ public class InterpreterTest {
     final FrameworkConfig config = Frameworks.newConfigBuilder()
         .lex(Lex.ORACLE)
         .defaultSchema(
-            OptiqAssert.addSchema(rootSchema, OptiqAssert.SchemaSpec.HR))
+            CalciteAssert.addSchema(rootSchema, CalciteAssert.SchemaSpec.HR))
         .build();
     planner = Frameworks.getPlanner(config);
   }
@@ -131,8 +129,8 @@ public class InterpreterTest {
         "[200, 20, Eric, 8000.0, 500]");
   }
 
-  /** Tests executing a plan on a {@link net.hydromatic.optiq.ScannableTable}
-   * using an interpreter. */
+  /** Tests executing a plan on a
+   * {@link org.apache.calcite.schema.ScannableTable} using an interpreter. */
   @Test public void testInterpretScannableTable() throws Exception {
     rootSchema.add("beatles", new ScannableTableTest.BeatlesTable());
     SqlNode parse =
@@ -151,7 +149,7 @@ public class InterpreterTest {
   }
 
   /** Tests executing a plan on a single-column
-   * {@link net.hydromatic.optiq.ScannableTable} using an interpreter. */
+   * {@link org.apache.calcite.schema.ScannableTable} using an interpreter. */
   @Test public void testInterpretSimpleScannableTable() throws Exception {
     rootSchema.add("simple", new ScannableTableTest.SimpleTable());
     SqlNode parse =

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/test/JdbcAdapterTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/JdbcAdapterTest.java b/core/src/test/java/org/apache/calcite/test/JdbcAdapterTest.java
index 4e4a7b8..3c2658c 100644
--- a/core/src/test/java/org/apache/calcite/test/JdbcAdapterTest.java
+++ b/core/src/test/java/org/apache/calcite/test/JdbcAdapterTest.java
@@ -14,30 +14,27 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.test;
+package org.apache.calcite.test;
 
 import org.junit.Test;
 
 /**
- * Tests for the {@code net.hydromatic.optiq.impl.jdbc} package.
+ * Tests for the {@code org.apache.calcite.adapter.jdbc} package.
  */
 public class JdbcAdapterTest {
   @Test public void testUnionPlan() {
-    OptiqAssert.that()
+    CalciteAssert.that()
         .withModel(JdbcTest.FOODMART_MODEL)
-        .query(
-            "select * from \"sales_fact_1997\"\n"
+        .query("select * from \"sales_fact_1997\"\n"
             + "union all\n"
             + "select * from \"sales_fact_1998\"")
-        .explainContains(
-            "PLAN=JdbcToEnumerableConverter\n"
-            + "  JdbcUnionRel(all=[true])\n"
+        .explainContains("PLAN=JdbcToEnumerableConverter\n"
+            + "  JdbcUnion(all=[true])\n"
             + "    JdbcTableScan(table=[[foodmart, sales_fact_1997]])\n"
             + "    JdbcTableScan(table=[[foodmart, sales_fact_1998]])")
         .runs()
-        .enable(OptiqAssert.CONNECTION_SPEC.url.startsWith("jdbc:hsqldb:"))
-        .planHasSql(
-            "SELECT *\n"
+        .enable(CalciteAssert.CONNECTION_SPEC.url.startsWith("jdbc:hsqldb:"))
+        .planHasSql("SELECT *\n"
             + "FROM \"foodmart\".\"sales_fact_1997\"\n"
             + "UNION ALL\n"
             + "SELECT *\n"
@@ -45,18 +42,16 @@ public class JdbcAdapterTest {
   }
 
   @Test public void testFilterUnionPlan() {
-    OptiqAssert.that()
+    CalciteAssert.that()
         .withModel(JdbcTest.FOODMART_MODEL)
-        .query(
-            "select * from (\n"
+        .query("select * from (\n"
             + "  select * from \"sales_fact_1997\"\n"
             + "  union all\n"
             + "  select * from \"sales_fact_1998\")\n"
             + "where \"product_id\" = 1")
         .runs()
-        .enable(OptiqAssert.CONNECTION_SPEC.url.startsWith("jdbc:hsqldb:"))
-        .planHasSql(
-            "SELECT *\n"
+        .enable(CalciteAssert.CONNECTION_SPEC.url.startsWith("jdbc:hsqldb:"))
+        .planHasSql("SELECT *\n"
             + "FROM \"foodmart\".\"sales_fact_1997\"\n"
             + "WHERE \"product_id\" = 1\n"
             + "UNION ALL\n"
@@ -66,19 +61,17 @@ public class JdbcAdapterTest {
   }
 
   @Test public void testInPlan() {
-    OptiqAssert.that()
+    CalciteAssert.that()
         .withModel(JdbcTest.FOODMART_MODEL)
-        .query(
-            "select \"store_id\", \"store_name\" from \"store\"\n"
+        .query("select \"store_id\", \"store_name\" from \"store\"\n"
             + "where \"store_name\" in ('Store 1', 'Store 10', 'Store 11', 'Store 15', 'Store 16', 'Store 24', 'Store 3', 'Store 7')")
         .runs()
-        .enable(OptiqAssert.CONNECTION_SPEC.url.startsWith("jdbc:hsqldb:"))
+        .enable(CalciteAssert.CONNECTION_SPEC.url.startsWith("jdbc:hsqldb:"))
         .planHasSql(
             "SELECT \"store_id\", \"store_name\"\n"
             + "FROM \"foodmart\".\"store\"\n"
             + "WHERE \"store_name\" = 'Store 1' OR \"store_name\" = 'Store 10' OR \"store_name\" = 'Store 11' OR \"store_name\" = 'Store 15' OR \"store_name\" = 'Store 16' OR \"store_name\" = 'Store 24' OR \"store_name\" = 'Store 3' OR \"store_name\" = 'Store 7'")
-        .returns(
-            "store_id=1; store_name=Store 1\n"
+        .returns("store_id=1; store_name=Store 1\n"
             + "store_id=3; store_name=Store 3\n"
             + "store_id=7; store_name=Store 7\n"
             + "store_id=10; store_name=Store 10\n"

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/test/JdbcFrontJdbcBackLinqMiddleTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/JdbcFrontJdbcBackLinqMiddleTest.java b/core/src/test/java/org/apache/calcite/test/JdbcFrontJdbcBackLinqMiddleTest.java
index 035563d..a9a2b3a 100644
--- a/core/src/test/java/org/apache/calcite/test/JdbcFrontJdbcBackLinqMiddleTest.java
+++ b/core/src/test/java/org/apache/calcite/test/JdbcFrontJdbcBackLinqMiddleTest.java
@@ -14,12 +14,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.test;
+package org.apache.calcite.test;
 
 import org.junit.Ignore;
 import org.junit.Test;
 
-import static net.hydromatic.optiq.test.OptiqAssert.that;
+import static org.apache.calcite.test.CalciteAssert.that;
 
 /**
  * Tests for a JDBC front-end and JDBC back-end where the processing is not
@@ -30,10 +30,9 @@ public class JdbcFrontJdbcBackLinqMiddleTest {
 
   @Test public void testTable() {
     that()
-        .with(OptiqAssert.Config.JDBC_FOODMART)
+        .with(CalciteAssert.Config.JDBC_FOODMART)
         .query("select * from \"foodmart\".\"days\"")
-        .returns(
-            "day=1; week_day=Sunday\n"
+        .returns("day=1; week_day=Sunday\n"
             + "day=2; week_day=Monday\n"
             + "day=5; week_day=Thursday\n"
             + "day=4; week_day=Wednesday\n"
@@ -44,21 +43,18 @@ public class JdbcFrontJdbcBackLinqMiddleTest {
 
   @Test public void testWhere() {
     that()
-        .with(OptiqAssert.Config.JDBC_FOODMART)
+        .with(CalciteAssert.Config.JDBC_FOODMART)
         .query("select * from \"foodmart\".\"days\" where \"day\" < 3")
-        .returns(
-            "day=1; week_day=Sunday\n"
+        .returns("day=1; week_day=Sunday\n"
             + "day=2; week_day=Monday\n");
   }
 
   @Test public void testWhere2() {
     that()
-        .with(OptiqAssert.Config.JDBC_FOODMART)
-        .query(
-            "select * from \"foodmart\".\"days\"\n"
+        .with(CalciteAssert.Config.JDBC_FOODMART)
+        .query("select * from \"foodmart\".\"days\"\n"
             + "where not (lower(\"week_day\") = 'wednesday')")
-        .returns(
-            "day=1; week_day=Sunday\n"
+        .returns("day=1; week_day=Sunday\n"
             + "day=2; week_day=Monday\n"
             + "day=5; week_day=Thursday\n"
             + "day=3; week_day=Tuesday\n"
@@ -68,9 +64,8 @@ public class JdbcFrontJdbcBackLinqMiddleTest {
 
   @Test public void testCase() {
     that()
-        .with(OptiqAssert.Config.JDBC_FOODMART)
-        .query(
-            "select \"day\",\n"
+        .with(CalciteAssert.Config.JDBC_FOODMART)
+        .query("select \"day\",\n"
             + " \"week_day\",\n"
             + " case when \"day\" < 3 then upper(\"week_day\")\n"
             + "      when \"day\" < 5 then lower(\"week_day\")\n"
@@ -78,8 +73,7 @@ public class JdbcFrontJdbcBackLinqMiddleTest {
             + "from \"foodmart\".\"days\"\n"
             + "where \"day\" <> 1\n"
             + "order by \"day\"")
-        .returns(
-            "day=2; week_day=Monday; D=MONDAY\n"
+        .returns("day=2; week_day=Monday; D=MONDAY\n"
             + "day=3; week_day=Tuesday; D=tuesday\n"
             + "day=4; week_day=Wednesday; D=wednesday\n"
             + "day=5; week_day=Thursday; D=Thursday\n"
@@ -89,9 +83,8 @@ public class JdbcFrontJdbcBackLinqMiddleTest {
 
   @Test public void testGroup() {
     that()
-        .with(OptiqAssert.Config.JDBC_FOODMART)
-        .query(
-            "select s, count(*) as c, min(\"week_day\") as mw from (\n"
+        .with(CalciteAssert.Config.JDBC_FOODMART)
+        .query("select s, count(*) as c, min(\"week_day\") as mw from (\n"
             + "select \"week_day\",\n"
             + "  substring(\"week_day\" from 1 for 1) as s\n"
             + "from \"foodmart\".\"days\")\n"
@@ -106,9 +99,8 @@ public class JdbcFrontJdbcBackLinqMiddleTest {
 
   @Test public void testGroupEmpty() {
     that()
-        .with(OptiqAssert.Config.JDBC_FOODMART)
-        .query(
-            "select count(*) as c\n"
+        .with(CalciteAssert.Config.JDBC_FOODMART)
+        .query("select count(*) as c\n"
             + "from \"foodmart\".\"days\"")
         .returns("C=7\n");
   }
@@ -122,30 +114,27 @@ public class JdbcFrontJdbcBackLinqMiddleTest {
   @Ignore("non-deterministic on JDK 1.7 vs 1.8")
   @Test public void testJoinTheta() {
     that()
-        .with(OptiqAssert.Config.FOODMART_CLONE)
-        .query(
-            "select count(*) from (\n"
+        .with(CalciteAssert.Config.FOODMART_CLONE)
+        .query("select count(*) from (\n"
             + "  select *\n"
             + "  from \"foodmart\".\"sales_fact_1997\" as s\n"
             + "  join \"foodmart\".\"customer\" as c\n"
             + "  on s.\"customer_id\" - c.\"customer_id\" = 0)")
-        .explainContains(
-            "EnumerableAggregateRel(group=[{}], EXPR$0=[COUNT()])\n"
-                + "  EnumerableCalcRel(expr#0..1=[{inputs}], expr#2=[0], expr#3=[-($t0, $t1)], expr#4=[=($t3, $t2)], DUMMY=[$t2], $condition=[$t4])\n"
-                + "    EnumerableJoinRel(condition=[true], joinType=[inner])\n"
-                + "      JdbcToEnumerableConverter\n"
-                + "        JdbcProjectRel(customer_id=[$2])\n"
-                + "          JdbcTableScan(table=[[foodmart, sales_fact_1997]])\n"
-                + "      JdbcToEnumerableConverter\n"
-                + "        JdbcProjectRel(customer_id=[$0])\n"
-                + "          JdbcTableScan(table=[[foodmart, customer]])");
+        .explainContains("EnumerableAggregate(group=[{}], EXPR$0=[COUNT()])\n"
+            + "  EnumerableCalc(expr#0..1=[{inputs}], expr#2=[0], expr#3=[-($t0, $t1)], expr#4=[=($t3, $t2)], DUMMY=[$t2], $condition=[$t4])\n"
+            + "    EnumerableJoin(condition=[true], joinType=[inner])\n"
+            + "      JdbcToEnumerableConverter\n"
+            + "        JdbcProject(customer_id=[$2])\n"
+            + "          JdbcTableScan(table=[[foodmart, sales_fact_1997]])\n"
+            + "      JdbcToEnumerableConverter\n"
+            + "        JdbcProject(customer_id=[$0])\n"
+            + "          JdbcTableScan(table=[[foodmart, customer]])");
   }
 
   @Test public void testJoinGroupByEmpty() {
     that()
-        .with(OptiqAssert.Config.JDBC_FOODMART)
-        .query(
-            "select count(*) from (\n"
+        .with(CalciteAssert.Config.JDBC_FOODMART)
+        .query("select count(*) from (\n"
             + "  select *\n"
             + "  from \"foodmart\".\"sales_fact_1997\" as s\n"
             + "  join \"foodmart\".\"customer\" as c\n"
@@ -155,30 +144,27 @@ public class JdbcFrontJdbcBackLinqMiddleTest {
 
   @Test public void testJoinGroupByOrderBy() {
     that()
-        .with(OptiqAssert.Config.JDBC_FOODMART)
-        .query(
-            "select count(*), c.\"state_province\", sum(s.\"unit_sales\") as s\n"
+        .with(CalciteAssert.Config.JDBC_FOODMART)
+        .query("select count(*), c.\"state_province\",\n"
+            + "  sum(s.\"unit_sales\") as s\n"
             + "from \"foodmart\".\"sales_fact_1997\" as s\n"
             + "  join \"foodmart\".\"customer\" as c\n"
             + "  on s.\"customer_id\" = c.\"customer_id\"\n"
             + "group by c.\"state_province\"\n"
             + "order by c.\"state_province\"")
-        .returns(
-            "EXPR$0=24442; state_province=CA; S=74748.0000\n"
+        .returns("EXPR$0=24442; state_province=CA; S=74748.0000\n"
             + "EXPR$0=21611; state_province=OR; S=67659.0000\n"
             + "EXPR$0=40784; state_province=WA; S=124366.0000\n");
   }
 
   @Test public void testCompositeGroupBy() {
     that()
-        .with(OptiqAssert.Config.JDBC_FOODMART)
-        .query(
-            "select count(*) as c, c.\"state_province\"\n"
+        .with(CalciteAssert.Config.JDBC_FOODMART)
+        .query("select count(*) as c, c.\"state_province\"\n"
             + "from \"foodmart\".\"customer\" as c\n"
             + "group by c.\"state_province\", c.\"country\"\n"
             + "order by c, 1")
-        .returns(
-            "C=78; state_province=Sinaloa\n"
+        .returns("C=78; state_province=Sinaloa\n"
             + "C=90; state_province=Oaxaca\n"
             + "C=93; state_province=Veracruz\n"
             + "C=97; state_province=Mexico\n"
@@ -201,9 +187,8 @@ public class JdbcFrontJdbcBackLinqMiddleTest {
     // Distinct count
     // Not all GROUP columns are projected
     that()
-        .with(OptiqAssert.Config.JDBC_FOODMART)
-        .query(
-            "select c.\"state_province\",\n"
+        .with(CalciteAssert.Config.JDBC_FOODMART)
+        .query("select c.\"state_province\",\n"
             + "  sum(s.\"unit_sales\") as s,\n"
             + "  count(distinct c.\"customer_id\") as dc\n"
             + "from \"foodmart\".\"sales_fact_1997\" as s\n"
@@ -211,15 +196,13 @@ public class JdbcFrontJdbcBackLinqMiddleTest {
             + "  on s.\"customer_id\" = c.\"customer_id\"\n"
             + "group by c.\"state_province\", c.\"country\"\n"
             + "order by c.\"state_province\", 2")
-        .planHasSql(
-            "SELECT `state_province`, `S`, `DC`\n"
+        .planHasSql("SELECT `state_province`, `S`, `DC`\n"
             + "FROM (SELECT `customer`.`state_province`, `customer`.`country`, SUM(`sales_fact_1997`.`unit_sales`) AS `S`, COUNT(DISTINCT `customer`.`customer_id`) AS `DC`\n"
             + "FROM `foodmart`.`sales_fact_1997`\n"
             + "INNER JOIN `foodmart`.`customer` ON `sales_fact_1997`.`customer_id` = `customer`.`customer_id`\n"
             + "GROUP BY `customer`.`state_province`, `customer`.`country`) AS `t0`\n"
             + "ORDER BY `state_province`, `S`")
-        .returns(
-            "state_province=CA; S=74748.0000; DC=2716\n"
+        .returns("state_province=CA; S=74748.0000; DC=2716\n"
             + "state_province=OR; S=67659.0000; DC=1037\n"
             + "state_province=WA; S=124366.0000; DC=1828\n");
   }
@@ -227,16 +210,14 @@ public class JdbcFrontJdbcBackLinqMiddleTest {
   @Ignore
   @Test public void testPlan() {
     that()
-        .with(OptiqAssert.Config.JDBC_FOODMART)
-        .query(
-            "select c.\"state_province\"\n"
+        .with(CalciteAssert.Config.JDBC_FOODMART)
+        .query("select c.\"state_province\"\n"
             + "from \"foodmart\".\"customer\" as c\n"
             + "where c.\"state_province\" = 'USA'")
-        .planContains(
-            "            public boolean moveNext() {\n"
+        .planContains("            public boolean moveNext() {\n"
             + "              while (inputEnumerator.moveNext()) {\n"
             + "                final String v = (String) ((Object[]) inputEnumerator.current())[10];\n"
-            + "                if (v != null && net.hydromatic.optiq.runtime.SqlFunctions.eq(v, \"USA\")) {\n"
+            + "                if (v != null && org.apache.calcite.runtime.SqlFunctions.eq(v, \"USA\")) {\n"
             + "                  return true;\n"
             + "                }\n"
             + "              }\n"
@@ -247,22 +228,20 @@ public class JdbcFrontJdbcBackLinqMiddleTest {
   @Ignore
   @Test public void testPlan2() {
     that()
-        .with(OptiqAssert.Config.JDBC_FOODMART)
+        .with(CalciteAssert.Config.JDBC_FOODMART)
         .withSchema("foodmart")
-        .query(
-            "select \"customer\".\"state_province\" as \"c0\",\n"
+        .query("select \"customer\".\"state_province\" as \"c0\",\n"
             + " \"customer\".\"country\" as \"c1\"\n"
             + "from \"customer\" as \"customer\"\n"
             + "where (\"customer\".\"country\" = 'USA')\n"
             + "and UPPER(\"customer\".\"state_province\") = UPPER('CA')\n"
             + "group by \"customer\".\"state_province\", \"customer\".\"country\"\n"
             + "order by \"customer\".\"state_province\" ASC")
-        .planContains(
-            "          public boolean moveNext() {\n"
+        .planContains("          public boolean moveNext() {\n"
             + "            while (inputEnumerator.moveNext()) {\n"
             + "              final Object[] current12 = (Object[]) inputEnumerator.current();\n"
             + "              final String v1 = (String) current12[10];\n"
-            + "              if (net.hydromatic.optiq.runtime.SqlFunctions.eq((String) current12[12], \"USA\") && (v1 != null && net.hydromatic.optiq.runtime.SqlFunctions.eq(net.hydromatic.optiq.runtime.SqlFunctions.upper(v1), net.hydromatic.optiq.runtime.SqlFunctions.trim(net.hydromatic.optiq.runtime.SqlFunctions.upper(\"CA\"))))) {\n"
+            + "              if (org.apache.calcite.runtime.SqlFunctions.eq((String) current12[12], \"USA\") && (v1 != null && org.apache.calcite.runtime.SqlFunctions.eq(org.apache.calcite.runtime.SqlFunctions.upper(v1), org.apache.calcite.runtime.SqlFunctions.trim(org.apache.calcite.runtime.SqlFunctions.upper(\"CA\"))))) {\n"
             + "                return true;\n"
             + "              }\n"
             + "            }\n"
@@ -277,11 +256,11 @@ public class JdbcFrontJdbcBackLinqMiddleTest {
     //
     // inventory_fact_1997 is on the LHS because it is larger than store.
     that()
-        .with(OptiqAssert.Config.FOODMART_CLONE)
+        .with(CalciteAssert.Config.FOODMART_CLONE)
         .query(
             "select \"store\".\"store_country\" as \"c0\", sum(\"inventory_fact_1997\".\"supply_time\") as \"m0\" from \"store\" as \"store\", \"inventory_fact_1997\" as \"inventory_fact_1997\" where \"inventory_fact_1997\".\"store_id\" = \"store\".\"store_id\" group by \"store\".\"store_country\"")
         .planContains(
-            "  final net.hydromatic.linq4j.Enumerable _inputEnumerable1 = left.join(right, new net.hydromatic.linq4j.function.Function1() {\n");
+            "  final org.apache.calcite.linq4j.Enumerable _inputEnumerable1 = left.join(right, new org.apache.calcite.linq4j.function.Function1() {\n");
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/test/JdbcFrontJdbcBackTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/JdbcFrontJdbcBackTest.java b/core/src/test/java/org/apache/calcite/test/JdbcFrontJdbcBackTest.java
index b8fb934..be071c6 100644
--- a/core/src/test/java/org/apache/calcite/test/JdbcFrontJdbcBackTest.java
+++ b/core/src/test/java/org/apache/calcite/test/JdbcFrontJdbcBackTest.java
@@ -14,9 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.test;
+package org.apache.calcite.test;
 
-import net.hydromatic.optiq.jdbc.OptiqConnection;
+import org.apache.calcite.jdbc.CalciteConnection;
 
 import com.google.common.base.Function;
 
@@ -26,9 +26,10 @@ import org.junit.Test;
 import java.sql.ResultSet;
 import java.sql.SQLException;
 
-import static net.hydromatic.optiq.test.OptiqAssert.that;
+import static org.apache.calcite.test.CalciteAssert.that;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 
 /**
  * Tests for a JDBC front-end and JDBC back-end.
@@ -42,20 +43,19 @@ import static org.junit.Assert.*;
 public class JdbcFrontJdbcBackTest {
   @Test public void testWhere2() {
     that()
-        .with(OptiqAssert.Config.JDBC_FOODMART)
+        .with(CalciteAssert.Config.JDBC_FOODMART)
         .query("select * from \"foodmart\".\"days\" where \"day\" < 3")
-        .returns(
-            "day=1; week_day=Sunday\n"
+        .returns("day=1; week_day=Sunday\n"
             + "day=2; week_day=Monday\n");
   }
 
   @Ignore
   @Test public void testTables() throws Exception {
     that()
-        .with(OptiqAssert.Config.JDBC_FOODMART)
+        .with(CalciteAssert.Config.JDBC_FOODMART)
         .doWithConnection(
-            new Function<OptiqConnection, Object>() {
-              public Object apply(OptiqConnection a0) {
+            new Function<CalciteConnection, Object>() {
+              public Object apply(CalciteConnection a0) {
                 try {
                   ResultSet rset =
                       a0.getMetaData().getTables(
@@ -77,10 +77,10 @@ public class JdbcFrontJdbcBackTest {
 
   @Test public void testTablesByType() throws Exception {
     that()
-        .with(OptiqAssert.Config.REGULAR_PLUS_METADATA)
+        .with(CalciteAssert.Config.REGULAR_PLUS_METADATA)
         .doWithConnection(
-            new Function<OptiqConnection, Object>() {
-              public Object apply(OptiqConnection a0) {
+            new Function<CalciteConnection, Object>() {
+              public Object apply(CalciteConnection a0) {
                 try {
                   ResultSet rset =
                       a0.getMetaData().getTables(
@@ -103,10 +103,10 @@ public class JdbcFrontJdbcBackTest {
 
   @Test public void testColumns() throws Exception {
     that()
-        .with(OptiqAssert.Config.JDBC_FOODMART)
+        .with(CalciteAssert.Config.JDBC_FOODMART)
         .doWithConnection(
-            new Function<OptiqConnection, Object>() {
-              public Object apply(OptiqConnection a0) {
+            new Function<CalciteConnection, Object>() {
+              public Object apply(CalciteConnection a0) {
                 try {
                   ResultSet rset =
                       a0.getMetaData().getColumns(
@@ -131,10 +131,10 @@ public class JdbcFrontJdbcBackTest {
    * empty result set. */
   @Test public void testEmpty() throws Exception {
     that()
-        .with(OptiqAssert.Config.JDBC_FOODMART)
+        .with(CalciteAssert.Config.JDBC_FOODMART)
         .doWithConnection(
-            new Function<OptiqConnection, Object>() {
-              public Object apply(OptiqConnection a0) {
+            new Function<CalciteConnection, Object>() {
+              public Object apply(CalciteConnection a0) {
                 try {
                   ResultSet rset =
                       a0.getMetaData().getPrimaryKeys(
@@ -150,16 +150,15 @@ public class JdbcFrontJdbcBackTest {
 
   @Test public void testCase() {
     that()
-        .with(OptiqAssert.Config.JDBC_FOODMART)
+        .with(CalciteAssert.Config.JDBC_FOODMART)
         .withSchema("foodmart")
-        .query(
-            "select case when \"sales_fact_1997\".\"promotion_id\" = 1 then 0\n"
-            + "                        else \"sales_fact_1997\".\"store_sales\" end as \"c0\"\n"
+        .query("select\n"
+            + "  case when \"sales_fact_1997\".\"promotion_id\" = 1 then 0\n"
+            + "  else \"sales_fact_1997\".\"store_sales\" end as \"c0\"\n"
             + "from \"sales_fact_1997\" as \"sales_fact_1997\""
             + "where \"product_id\" = 1\n"
             + "and \"time_id\" < 400")
-        .returns(
-            "c0=11.4000\n"
+        .returns("c0=11.4000\n"
             + "c0=8.5500\n");
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/test/JdbcFrontLinqBackTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/JdbcFrontLinqBackTest.java b/core/src/test/java/org/apache/calcite/test/JdbcFrontLinqBackTest.java
index 587fd24..4f1edc0 100644
--- a/core/src/test/java/org/apache/calcite/test/JdbcFrontLinqBackTest.java
+++ b/core/src/test/java/org/apache/calcite/test/JdbcFrontLinqBackTest.java
@@ -14,21 +14,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.test;
+package org.apache.calcite.test;
 
-import net.hydromatic.linq4j.Enumerator;
-import net.hydromatic.linq4j.Linq4j;
-import net.hydromatic.linq4j.QueryProvider;
-import net.hydromatic.linq4j.Queryable;
-import net.hydromatic.linq4j.expressions.Expression;
-
-import net.hydromatic.optiq.*;
-import net.hydromatic.optiq.impl.*;
-import net.hydromatic.optiq.impl.java.JavaTypeFactory;
-import net.hydromatic.optiq.jdbc.OptiqConnection;
-
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.reltype.RelDataTypeFactory;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.jdbc.CalciteConnection;
+import org.apache.calcite.linq4j.Enumerator;
+import org.apache.calcite.linq4j.Linq4j;
+import org.apache.calcite.linq4j.QueryProvider;
+import org.apache.calcite.linq4j.Queryable;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.schema.Schemas;
+import org.apache.calcite.schema.impl.AbstractSchema;
+import org.apache.calcite.schema.impl.AbstractTableQueryable;
 
 import org.junit.Ignore;
 import org.junit.Test;
@@ -39,7 +39,7 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
 
-import static net.hydromatic.optiq.test.OptiqAssert.that;
+import static org.apache.calcite.test.CalciteAssert.that;
 
 /**
  * Tests for a JDBC front-end (with some quite complex SQL) and Linq4j back-end
@@ -51,12 +51,10 @@ public class JdbcFrontLinqBackTest {
    */
   @Test public void testSelect() {
     that()
-        .query(
-            "select *\n"
+        .query("select *\n"
             + "from \"foodmart\".\"sales_fact_1997\" as s\n"
             + "where s.\"cust_id\" = 100")
-        .returns(
-            "cust_id=100; prod_id=10\n");
+        .returns("cust_id=100; prod_id=10\n");
   }
 
   /**
@@ -64,13 +62,12 @@ public class JdbcFrontLinqBackTest {
    */
   @Test public void testJoin() {
     that()
-        .query(
-            "select *\n"
+        .query("select *\n"
             + "from \"foodmart\".\"sales_fact_1997\" as s\n"
             + "join \"hr\".\"emps\" as e\n"
             + "on e.\"empid\" = s.\"cust_id\"")
-        .returns(
-            "cust_id=100; prod_id=10; empid=100; deptno=10; name=Bill; salary=10000.0; commission=1000\n"
+        .returns(""
+            + "cust_id=100; prod_id=10; empid=100; deptno=10; name=Bill; salary=10000.0; commission=1000\n"
             + "cust_id=150; prod_id=20; empid=150; deptno=10; name=Sebastian; salary=7000.0; commission=null\n");
   }
 
@@ -79,12 +76,10 @@ public class JdbcFrontLinqBackTest {
    */
   @Test public void testGroupBy() {
     that()
-        .query(
-            "select \"deptno\", sum(\"empid\") as s, count(*) as c\n"
+        .query("select \"deptno\", sum(\"empid\") as s, count(*) as c\n"
             + "from \"hr\".\"emps\" as e\n"
             + "group by \"deptno\"")
-        .returns(
-            "deptno=20; S=200; C=1\n"
+        .returns("deptno=20; S=200; C=1\n"
             + "deptno=10; S=360; C=3\n");
   }
 
@@ -93,12 +88,10 @@ public class JdbcFrontLinqBackTest {
    */
   @Test public void testOrderBy() {
     that()
-        .query(
-            "select upper(\"name\") as un, \"deptno\"\n"
+        .query("select upper(\"name\") as un, \"deptno\"\n"
             + "from \"hr\".\"emps\" as e\n"
             + "order by \"deptno\", \"name\" desc")
-        .returns(
-            "UN=THEODORE; deptno=10\n"
+        .returns("UN=THEODORE; deptno=10\n"
             + "UN=SEBASTIAN; deptno=10\n"
             + "UN=BILL; deptno=10\n"
             + "UN=ERIC; deptno=20\n");
@@ -112,15 +105,13 @@ public class JdbcFrontLinqBackTest {
    */
   @Test public void testUnionAllOrderBy() {
     that()
-        .query(
-            "select \"name\"\n"
+        .query("select \"name\"\n"
             + "from \"hr\".\"emps\" as e\n"
             + "union all\n"
             + "select \"name\"\n"
             + "from \"hr\".\"depts\"\n"
             + "order by 1 desc")
-        .returns(
-            "name=Theodore\n"
+        .returns("name=Theodore\n"
             + "name=Sebastian\n"
             + "name=Sales\n"
             + "name=Marketing\n"
@@ -134,8 +125,7 @@ public class JdbcFrontLinqBackTest {
    */
   @Test public void testUnion() {
     that()
-        .query(
-            "select substring(\"name\" from 1 for 1) as x\n"
+        .query("select substring(\"name\" from 1 for 1) as x\n"
             + "from \"hr\".\"emps\" as e\n"
             + "union\n"
             + "select substring(\"name\" from 1 for 1) as y\n"
@@ -155,14 +145,12 @@ public class JdbcFrontLinqBackTest {
   @Ignore
   @Test public void testIntersect() {
     that()
-        .query(
-            "select substring(\"name\" from 1 for 1) as x\n"
+        .query("select substring(\"name\" from 1 for 1) as x\n"
             + "from \"hr\".\"emps\" as e\n"
             + "intersect\n"
             + "select substring(\"name\" from 1 for 1) as y\n"
             + "from \"hr\".\"depts\"")
-        .returns(
-            "X=S\n");
+        .returns("X=S\n");
   }
 
   /**
@@ -171,8 +159,7 @@ public class JdbcFrontLinqBackTest {
   @Ignore
   @Test public void testExcept() {
     that()
-        .query(
-            "select substring(\"name\" from 1 for 1) as x\n"
+        .query("select substring(\"name\" from 1 for 1) as x\n"
             + "from \"hr\".\"emps\" as e\n"
             + "except\n"
             + "select substring(\"name\" from 1 for 1) as y\n"
@@ -185,8 +172,7 @@ public class JdbcFrontLinqBackTest {
 
   @Test public void testWhereBad() {
     that()
-        .query(
-            "select *\n"
+        .query("select *\n"
             + "from \"foodmart\".\"sales_fact_1997\" as s\n"
             + "where empid > 120")
         .throws_("Column 'EMPID' not found in any table");
@@ -197,8 +183,7 @@ public class JdbcFrontLinqBackTest {
    * "RexToLixTranslator not incrementing local variable name counter". */
   @Test public void testWhereOr() {
     that()
-        .query(
-            "select * from \"hr\".\"emps\"\n"
+        .query("select * from \"hr\".\"emps\"\n"
             + "where (\"empid\" = 100 or \"empid\" = 200)\n"
             + "and \"deptno\" = 10")
         .returns(
@@ -207,12 +192,11 @@ public class JdbcFrontLinqBackTest {
 
   @Test public void testWhereLike() {
     that()
-        .query(
-            "select *\n"
+        .query("select *\n"
             + "from \"hr\".\"emps\" as e\n"
             + "where e.\"empid\" < 120 or e.\"name\" like 'S%'")
-        .returns(
-            "empid=100; deptno=10; name=Bill; salary=10000.0; commission=1000\n"
+        .returns(""
+            + "empid=100; deptno=10; name=Bill; salary=10000.0; commission=1000\n"
             + "empid=150; deptno=10; name=Sebastian; salary=7000.0; commission=null\n"
             + "empid=110; deptno=10; name=Theodore; salary=11500.0; commission=250\n");
   }
@@ -220,7 +204,7 @@ public class JdbcFrontLinqBackTest {
   @Test public void testInsert() {
     final List<JdbcTest.Employee> employees =
         new ArrayList<JdbcTest.Employee>();
-    OptiqAssert.AssertThat with = mutable(employees);
+    CalciteAssert.AssertThat with = mutable(employees);
     with.query("select * from \"foo\".\"bar\"")
         .returns(
             "empid=0; deptno=0; name=first; salary=0.0; commission=null\n");
@@ -228,12 +212,10 @@ public class JdbcFrontLinqBackTest {
         .returns("ROWCOUNT=4\n");
     with.query("select count(*) as c from \"foo\".\"bar\"")
         .returns("C=5\n");
-    with.query(
-        "insert into \"foo\".\"bar\" "
+    with.query("insert into \"foo\".\"bar\" "
         + "select * from \"hr\".\"emps\" where \"deptno\" = 10")
         .returns("ROWCOUNT=3\n");
-    with.query(
-        "select \"name\", count(*) as c from \"foo\".\"bar\" "
+    with.query("select \"name\", count(*) as c from \"foo\".\"bar\" "
         + "group by \"name\"")
         .returnsUnordered(
             "name=Bill; C=2",
@@ -243,19 +225,19 @@ public class JdbcFrontLinqBackTest {
             "name=Sebastian; C=2");
   }
 
-  private OptiqAssert.AssertThat mutable(
+  private CalciteAssert.AssertThat mutable(
       final List<JdbcTest.Employee> employees) {
     employees.add(new JdbcTest.Employee(0, 0, "first", 0f, null));
     return that()
         .with(
-            new OptiqAssert.ConnectionFactory() {
-              public OptiqConnection createConnection() throws Exception {
+            new CalciteAssert.ConnectionFactory() {
+              public CalciteConnection createConnection() throws Exception {
                 final Connection connection =
-                    OptiqAssert.getConnection("hr", "foodmart");
-                OptiqConnection optiqConnection = connection.unwrap(
-                    OptiqConnection.class);
+                    CalciteAssert.getConnection("hr", "foodmart");
+                CalciteConnection calciteConnection = connection.unwrap(
+                    CalciteConnection.class);
                 SchemaPlus rootSchema =
-                    optiqConnection.getRootSchema();
+                    calciteConnection.getRootSchema();
                 SchemaPlus mapSchema =
                     rootSchema.add("foo", new AbstractSchema());
                 final String tableName = "bar";
@@ -294,7 +276,7 @@ public class JdbcFrontLinqBackTest {
                       }
                     };
                 mapSchema.add(tableName, table);
-                return optiqConnection;
+                return calciteConnection;
               }
             });
   }
@@ -302,11 +284,10 @@ public class JdbcFrontLinqBackTest {
   @Test public void testInsert2() {
     final List<JdbcTest.Employee> employees =
         new ArrayList<JdbcTest.Employee>();
-    OptiqAssert.AssertThat with = mutable(employees);
+    CalciteAssert.AssertThat with = mutable(employees);
     with.query("insert into \"foo\".\"bar\" values (1, 1, 'second', 2, 2)")
         .returns("ROWCOUNT=1\n");
-    with.query(
-        "insert into \"foo\".\"bar\"\n"
+    with.query("insert into \"foo\".\"bar\"\n"
         + "values (1, 3, 'third', 0, 3), (1, 4, 'fourth', 0, 4), (1, 5, 'fifth ', 0, 3)")
         .returns("ROWCOUNT=3\n");
     with.query("select count(*) as c from \"foo\".\"bar\"")
@@ -321,9 +302,8 @@ public class JdbcFrontLinqBackTest {
   @Test public void testInsertMultipleRowMismatch() {
     final List<JdbcTest.Employee> employees =
         new ArrayList<JdbcTest.Employee>();
-    OptiqAssert.AssertThat with = mutable(employees);
-    with.query(
-        "insert into \"foo\".\"bar\" values\n"
+    CalciteAssert.AssertThat with = mutable(employees);
+    with.query("insert into \"foo\".\"bar\" values\n"
         + " (1, 3, 'third'),\n"
         + " (1, 4, 'fourth'),\n"
         + " (1, 5, 'fifth ', 3)")


[32/58] [abbrv] [partial] incubator-calcite git commit: [CALCITE-306] Standardize code style for "import package.*; "

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/rules/LoptJoinTree.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/LoptJoinTree.java b/core/src/main/java/org/apache/calcite/rel/rules/LoptJoinTree.java
index 0ac7197..55e41c7 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/LoptJoinTree.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/LoptJoinTree.java
@@ -14,18 +14,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
+package org.apache.calcite.rel.rules;
 
-import java.util.*;
-
-import org.eigenbase.rel.*;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Join;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 
+import java.util.List;
+
 /**
- * Utility class used to store a {@link JoinRelBase} tree and the factors that
- * make up the tree.
+ * Utility class used to store a {@link org.apache.calcite.rel.core.Join} tree
+ * and the factors that make up the tree.
  *
  * <p>Because {@link RelNode}s can be duplicated in a query
  * when you have a self-join, factor ids are needed to distinguish between the
@@ -115,7 +116,7 @@ public class LoptJoinTree {
   public LoptJoinTree getLeft() {
     final Node node = (Node) factorTree;
     return new LoptJoinTree(
-        ((JoinRelBase) joinTree).getLeft(),
+        ((Join) joinTree).getLeft(),
         node.getLeft(),
         node.getLeft().getParent().isRemovableSelfJoin());
   }
@@ -123,7 +124,7 @@ public class LoptJoinTree {
   public LoptJoinTree getRight() {
     final Node node = (Node) factorTree;
     return new LoptJoinTree(
-        ((JoinRelBase) joinTree).getRight(),
+        ((Join) joinTree).getRight(),
         node.getRight(),
         node.getRight().getParent().isRemovableSelfJoin());
   }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/rules/LoptMultiJoin.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/LoptMultiJoin.java b/core/src/main/java/org/apache/calcite/rel/rules/LoptMultiJoin.java
index 8410894..071dbc1 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/LoptMultiJoin.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/LoptMultiJoin.java
@@ -14,59 +14,71 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
-
-import java.util.*;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.rel.metadata.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.rex.*;
-import org.eigenbase.sql.SqlKind;
-import org.eigenbase.util.IntList;
-
-import net.hydromatic.optiq.util.BitSets;
+package org.apache.calcite.rel.rules;
+
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.core.SemiJoin;
+import org.apache.calcite.rel.metadata.RelColumnOrigin;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.IntList;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.Map;
+import java.util.Set;
+
 /**
  * Utility class that keeps track of the join factors that
- * make up a {@link MultiJoinRel}.
+ * make up a {@link MultiJoin}.
  */
 public class LoptMultiJoin {
   //~ Instance fields --------------------------------------------------------
 
   /**
-   * The MultiJoinRel being optimized
+   * The MultiJoin being optimized
    */
-  MultiJoinRel multiJoin;
+  MultiJoin multiJoin;
 
   /**
-   * Join filters associated with the MultiJoinRel, decomposed into a list.
+   * Join filters associated with the MultiJoin, decomposed into a list.
    * Excludes left/right outer join filters.
    */
   private List<RexNode> joinFilters;
 
   /**
-   * All join filters associated with the MultiJoinRel, decomposed into a
+   * All join filters associated with the MultiJoin, decomposed into a
    * list. Includes left/right outer join filters.
    */
   private List<RexNode> allJoinFilters;
 
   /**
-   * Number of factors into the MultiJoinRel
+   * Number of factors into the MultiJoin
    */
   private final int nJoinFactors;
 
   /**
-   * Total number of fields in the MultiJoinRel
+   * Total number of fields in the MultiJoin
    */
   private int nTotalFields;
 
   /**
-   * Original inputs into the MultiJoinRel
+   * Original inputs into the MultiJoin
    */
   private final ImmutableList<RelNode> joinFactors;
 
@@ -150,7 +162,7 @@ public class LoptMultiJoin {
   /**
    * The semijoins that allow the join of a dimension table to be removed
    */
-  SemiJoinRel [] joinRemovalSemiJoins;
+  SemiJoin[] joinRemovalSemiJoins;
 
   /**
    * Set of null-generating factors whose corresponding outer join can be
@@ -168,7 +180,7 @@ public class LoptMultiJoin {
 
   //~ Constructors -----------------------------------------------------------
 
-  public LoptMultiJoin(MultiJoinRel multiJoin) {
+  public LoptMultiJoin(MultiJoin multiJoin) {
     this.multiJoin = multiJoin;
     joinFactors = ImmutableList.copyOf(multiJoin.getInputs());
     nJoinFactors = joinFactors.size();
@@ -219,7 +231,7 @@ public class LoptMultiJoin {
     factory = multiJoin.getCluster().getTypeFactory();
 
     joinRemovalFactors = new Integer[nJoinFactors];
-    joinRemovalSemiJoins = new SemiJoinRel[nJoinFactors];
+    joinRemovalSemiJoins = new SemiJoin[nJoinFactors];
 
     removableOuterJoinFactors = new HashSet<Integer>();
     removableSelfJoinPairs = new HashMap<Integer, RemovableSelfJoin>();
@@ -228,9 +240,9 @@ public class LoptMultiJoin {
   //~ Methods ----------------------------------------------------------------
 
   /**
-   * @return the MultiJoinRel corresponding to this multijoin
+   * @return the MultiJoin corresponding to this multijoin
    */
-  public MultiJoinRel getMultiJoinRel() {
+  public MultiJoin getMultiJoinRel() {
     return multiJoin;
   }
 
@@ -391,7 +403,7 @@ public class LoptMultiJoin {
    * @return the semijoin that allows the join of a dimension table to be
    * removed
    */
-  public SemiJoinRel getJoinRemovalSemiJoin(int dimIdx) {
+  public SemiJoin getJoinRemovalSemiJoin(int dimIdx) {
     return joinRemovalSemiJoins[dimIdx];
   }
 
@@ -413,7 +425,7 @@ public class LoptMultiJoin {
    * @param dimIdx id of the dimension factor
    * @param semiJoin the semijoin
    */
-  public void setJoinRemovalSemiJoin(int dimIdx, SemiJoinRel semiJoin) {
+  public void setJoinRemovalSemiJoin(int dimIdx, SemiJoin semiJoin) {
     joinRemovalSemiJoins[dimIdx] = semiJoin;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/rules/LoptOptimizeJoinRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/LoptOptimizeJoinRule.java b/core/src/main/java/org/apache/calcite/rel/rules/LoptOptimizeJoinRule.java
index f26c817..2f5643b 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/LoptOptimizeJoinRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/LoptOptimizeJoinRule.java
@@ -14,30 +14,57 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
-
-import java.util.*;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.rel.metadata.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.rex.*;
-import org.eigenbase.sql.fun.*;
-import org.eigenbase.util.ImmutableIntList;
-import org.eigenbase.util.Pair;
-import org.eigenbase.util.mapping.IntPair;
-
-import net.hydromatic.optiq.util.BitSets;
+package org.apache.calcite.rel.rules;
+
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.JoinInfo;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.RelFactories;
+import org.apache.calcite.rel.core.SemiJoin;
+import org.apache.calcite.rel.metadata.RelColumnOrigin;
+import org.apache.calcite.rel.metadata.RelMdUtil;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.ImmutableIntList;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.mapping.IntPair;
 
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Lists;
 
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
 /**
  * Planner rule that implements the heuristic planner for determining optimal
  * join orderings.
  *
- * <p>It is triggered by the pattern {@link ProjectRel} ({@link MultiJoinRel}).
+ * <p>It is triggered by the pattern
+ * {@link org.apache.calcite.rel.logical.LogicalProject}
+ * ({@link MultiJoin}).
  */
 public class LoptOptimizeJoinRule extends RelOptRule {
   public static final LoptOptimizeJoinRule INSTANCE =
@@ -55,7 +82,7 @@ public class LoptOptimizeJoinRule extends RelOptRule {
       RelFactories.JoinFactory joinFactory,
       RelFactories.ProjectFactory projectFactory,
       RelFactories.FilterFactory filterFactory) {
-    super(operand(MultiJoinRel.class, any()));
+    super(operand(MultiJoin.class, any()));
     this.joinFactory = joinFactory;
     this.projectFactory = projectFactory;
     this.filterFactory = filterFactory;
@@ -65,7 +92,7 @@ public class LoptOptimizeJoinRule extends RelOptRule {
 
   // implement RelOptRule
   public void onMatch(RelOptRuleCall call) {
-    final MultiJoinRel multiJoinRel = call.rel(0);
+    final MultiJoin multiJoinRel = call.rel(0);
     final LoptMultiJoin multiJoin = new LoptMultiJoin(multiJoinRel);
 
     findRemovableOuterJoins(multiJoin);
@@ -509,8 +536,8 @@ public class LoptOptimizeJoinRule extends RelOptRule {
                 newOffset));
       }
     }
-    ProjectRelBase newProject =
-        (ProjectRelBase) projectFactory.createProject(
+    Project newProject =
+        (Project) projectFactory.createProject(
             joinTree.getJoinTree(),
             newProjExprs,
             fieldNames);
@@ -798,15 +825,15 @@ public class LoptOptimizeJoinRule extends RelOptRule {
   }
 
   /**
-   * Returns true if a relnode corresponds to a JoinRel that wasn't one of the
-   * original MultiJoinRel input factors
+   * Returns whether a RelNode corresponds to a Join that wasn't one of the
+   * original MultiJoin input factors.
    */
   private boolean isJoinTree(RelNode rel) {
     // full outer joins were already optimized in a prior instantiation
     // of this rule; therefore we should never see a join input that's
     // a full outer join
-    if (rel instanceof JoinRelBase) {
-      assert ((JoinRelBase) rel).getJoinType() != JoinRelType.FULL;
+    if (rel instanceof Join) {
+      assert ((Join) rel).getJoinType() != JoinRelType.FULL;
       return true;
     } else {
       return false;
@@ -949,10 +976,10 @@ public class LoptOptimizeJoinRule extends RelOptRule {
     // The width cost is the width of the tree itself plus the widths
     // of its children.  Hence, skinnier rows are better when they're
     // lower in the tree since the width of a RelNode contributes to
-    // the cost of each JoinRel that appears above that RelNode.
+    // the cost of each LogicalJoin that appears above that RelNode.
     int width = tree.getRowType().getFieldCount();
     if (isJoinTree(tree)) {
-      JoinRelBase joinRel = (JoinRelBase) tree;
+      Join joinRel = (Join) tree;
       width +=
           rowWidthCost(joinRel.getLeft())
               + rowWidthCost(joinRel.getRight());
@@ -993,7 +1020,7 @@ public class LoptOptimizeJoinRule extends RelOptRule {
     int childNo = -1;
     LoptJoinTree left = joinTree.getLeft();
     LoptJoinTree right = joinTree.getRight();
-    JoinRelBase joinRel = (JoinRelBase) joinTree.getJoinTree();
+    Join joinRel = (Join) joinTree.getJoinTree();
     JoinRelType joinType = joinRel.getJoinType();
 
     // can't push factors pass self-joins because in order to later remove
@@ -1065,7 +1092,7 @@ public class LoptOptimizeJoinRule extends RelOptRule {
     // pushdown of the new factor as well as any swapping that may have
     // been done during the pushdown
     RexNode newCondition =
-        ((JoinRelBase) joinTree.getJoinTree()).getCondition();
+        ((Join) joinTree.getJoinTree()).getCondition();
     newCondition =
         adjustFilter(
             multiJoin,
@@ -1142,7 +1169,7 @@ public class LoptOptimizeJoinRule extends RelOptRule {
     // as a left outer join since it's being added to the RHS side of
     // the join; createJoinSubTree may swap the inputs and therefore
     // convert the left outer join to a right outer join; if the original
-    // MultiJoinRel was a full outer join, these should be the only
+    // MultiJoin was a full outer join, these should be the only
     // factors in the join, so create the join as a full outer join
     JoinRelType joinType;
     if (multiJoin.getMultiJoinRel().isFullOuterJoin()) {
@@ -1510,7 +1537,7 @@ public class LoptOptimizeJoinRule extends RelOptRule {
         multiJoin.getJoinFactor(dimIdx).getRowType().getFieldList();
     int nDimFields = dimFields.size();
     Integer [] replacementKeys = new Integer[nDimFields];
-    SemiJoinRel semiJoin = multiJoin.getJoinRemovalSemiJoin(dimIdx);
+    SemiJoin semiJoin = multiJoin.getJoinRemovalSemiJoin(dimIdx);
     ImmutableIntList dimKeys = semiJoin.getRightKeys();
     ImmutableIntList factKeys = semiJoin.getLeftKeys();
     for (int i = 0; i < dimKeys.size(); i++) {
@@ -1614,8 +1641,8 @@ public class LoptOptimizeJoinRule extends RelOptRule {
       }
       projects.add(Pair.of(projExpr, newFields.get(i).getName()));
     }
-    ProjectRelBase projRel =
-        (ProjectRelBase) projectFactory.createProject(
+    Project projRel =
+        (Project) projectFactory.createProject(
             currJoinRel,
             Pair.left(projects),
             Pair.right(projects));
@@ -1637,7 +1664,7 @@ public class LoptOptimizeJoinRule extends RelOptRule {
 
     // Filters referencing factors other than leftIdx and factorToAdd
     // still do need to be applied.  So, add them into a separate
-    // FilterRel placed on top off the projection created above.
+    // LogicalFilter placed on top off the projection created above.
     RelNode topRelNode = projRel;
     if (leftIdx >= 0) {
       topRelNode =
@@ -1661,7 +1688,7 @@ public class LoptOptimizeJoinRule extends RelOptRule {
   }
 
   /**
-   * Creates a JoinRel given left and right operands and a join condition.
+   * Creates a LogicalJoin given left and right operands and a join condition.
    * Swaps the operands if beneficial.
    *
    * @param multiJoin join factors being optimized
@@ -1674,11 +1701,11 @@ public class LoptOptimizeJoinRule extends RelOptRule {
    * otherwise, the condition has already been partially adjusted and only
    * needs to be further adjusted if swapping is done
    * @param filtersToAdd additional filters that may be added on top of the
-   * resulting JoinRel, if the join is a left or right outer join
+   * resulting LogicalJoin, if the join is a left or right outer join
    * @param selfJoin true if the join being created is a self-join that's
    * removable
    *
-   * @return created JoinRel
+   * @return created LogicalJoin
    */
   private LoptJoinTree createJoinSubtree(
       LoptMultiJoin multiJoin,
@@ -1960,7 +1987,7 @@ public class LoptOptimizeJoinRule extends RelOptRule {
    *
    * @return true if the join is removable
    */
-  public static boolean isRemovableSelfJoin(JoinRelBase joinRel) {
+  public static boolean isRemovableSelfJoin(Join joinRel) {
     final RelNode left = joinRel.getLeft();
     final RelNode right = joinRel.getRight();
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/rules/LoptSemiJoinOptimizer.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/LoptSemiJoinOptimizer.java b/core/src/main/java/org/apache/calcite/rel/rules/LoptSemiJoinOptimizer.java
index 319b323..2f95c20 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/LoptSemiJoinOptimizer.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/LoptSemiJoinOptimizer.java
@@ -14,22 +14,40 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
-
-import java.util.*;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.rel.metadata.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.rex.*;
-import org.eigenbase.sql.SqlKind;
-import org.eigenbase.sql.fun.*;
-import org.eigenbase.util.ImmutableIntList;
-
-import net.hydromatic.optiq.util.BitSets;
+package org.apache.calcite.rel.rules;
+
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.JoinInfo;
+import org.apache.calcite.rel.core.SemiJoin;
+import org.apache.calcite.rel.metadata.RelColumnOrigin;
+import org.apache.calcite.rel.metadata.RelMdUtil;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.ImmutableIntList;
 
 import com.google.common.collect.Lists;
 
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.BitSet;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.Map;
+import java.util.Set;
+
 /**
  * Implements the logic for determining the optimal
  * semi-joins to be used in processing joins in a query.
@@ -56,10 +74,10 @@ public class LoptSemiJoinOptimizer {
   /**
    * Associates potential semijoins with each fact table factor. The first
    * parameter in the map corresponds to the fact table. The second
-   * corresponds to the dimension table and a SemiJoinRel that captures all
+   * corresponds to the dimension table and a SemiJoin that captures all
    * the necessary semijoin data between that fact and dimension table
    */
-  private Map<Integer, Map<Integer, SemiJoinRel>> possibleSemiJoins;
+  private Map<Integer, Map<Integer, SemiJoin>> possibleSemiJoins;
 
   private final Comparator<Integer> factorCostComparator =
       new FactorCostComparator();
@@ -91,7 +109,7 @@ public class LoptSemiJoinOptimizer {
    * @param multiJoin join factors being optimized
    */
   public void makePossibleSemiJoins(LoptMultiJoin multiJoin) {
-    possibleSemiJoins = new HashMap<Integer, Map<Integer, SemiJoinRel>>();
+    possibleSemiJoins = new HashMap<Integer, Map<Integer, SemiJoin>>();
 
     // semijoins can't be used with any type of outer join, including full
     if (multiJoin.getMultiJoinRel().isFullOuterJoin()) {
@@ -102,8 +120,8 @@ public class LoptSemiJoinOptimizer {
     for (int factIdx = 0; factIdx < nJoinFactors; factIdx++) {
       Map<Integer, List<RexNode>> dimFilters =
           new HashMap<Integer, List<RexNode>>();
-      Map<Integer, SemiJoinRel> semiJoinMap =
-          new HashMap<Integer, SemiJoinRel>();
+      Map<Integer, SemiJoin> semiJoinMap =
+          new HashMap<Integer, SemiJoin>();
 
       // loop over all filters and find equality filters that reference
       // this factor and one other factor
@@ -137,7 +155,7 @@ public class LoptSemiJoinOptimizer {
       for (Integer dimIdx : dimIdxes) {
         List<RexNode> joinFilters = dimFilters.get(dimIdx);
         if (joinFilters != null) {
-          SemiJoinRel semiJoin =
+          SemiJoin semiJoin =
               findSemiJoinIndexByCost(
                   multiJoin,
                   joinFilters,
@@ -214,15 +232,15 @@ public class LoptSemiJoinOptimizer {
    * @param factIdx index in join factors corresponding to the fact table
    * @param dimIdx index in join factors corresponding to the dimension table
    *
-   * @return SemiJoinRel containing information regarding the semijoin that
+   * @return SemiJoin containing information regarding the semijoin that
    * can be used to filter the fact table
    */
-  private SemiJoinRel findSemiJoinIndexByCost(
+  private SemiJoin findSemiJoinIndexByCost(
       LoptMultiJoin multiJoin,
       List<RexNode> joinFilters,
       int factIdx,
       int dimIdx) {
-    // create a SemiJoinRel with the semi-join condition and keys
+    // create a SemiJoin with the semi-join condition and keys
     RexNode semiJoinCondition =
         RexUtil.composeConjunction(rexBuilder, joinFilters, true);
 
@@ -263,8 +281,8 @@ public class LoptSemiJoinOptimizer {
 
     // find the best index
     List<Integer> bestKeyOrder = new ArrayList<Integer>();
-    LcsRowScanRel tmpFactRel =
-        (LcsRowScanRel) factTable.toRel(
+    LcsTableScan tmpFactRel =
+        (LcsTableScan) factTable.toRel(
             RelOptUtil.getContext(factRel.getCluster()));
 
     LcsIndexOptimizer indexOptimizer = new LcsIndexOptimizer(tmpFactRel);
@@ -302,8 +320,8 @@ public class LoptSemiJoinOptimizer {
               multiJoin.getNumFieldsInJoinFactor(factIdx),
               semiJoinCondition);
     }
-    SemiJoinRel semiJoin =
-        new SemiJoinRel(
+    SemiJoin semiJoin =
+        new SemiJoin(
             factRel.getCluster(),
             factRel.getCluster().traitSetOf(Convention.NONE),
             factRel,
@@ -532,7 +550,7 @@ public class LoptSemiJoinOptimizer {
     for (int i = 0; i < nJoinFactors; i++) {
       Integer factIdx = sortedFactors[i];
       RelNode factRel = chosenSemiJoins[factIdx];
-      Map<Integer, SemiJoinRel> possibleDimensions =
+      Map<Integer, SemiJoin> possibleDimensions =
           possibleSemiJoins.get(factIdx);
       if (possibleDimensions == null) {
         continue;
@@ -545,7 +563,7 @@ public class LoptSemiJoinOptimizer {
       // fact table
       Set<Integer> dimIdxes = possibleDimensions.keySet();
       for (Integer dimIdx : dimIdxes) {
-        SemiJoinRel semiJoin = possibleDimensions.get(dimIdx);
+        SemiJoin semiJoin = possibleDimensions.get(dimIdx);
         if (semiJoin == null) {
           continue;
         }
@@ -566,13 +584,13 @@ public class LoptSemiJoinOptimizer {
       // if a suitable dimension table has been found, associate it
       // with the fact table in the chosenSemiJoins array; also remove
       // the entry from possibleSemiJoins so we won't chose it again;
-      // note that we create the SemiJoinRel using the chosen semijoins
+      // note that we create the SemiJoin using the chosen semijoins
       // already created for each factor so any chaining of filters will
       // be accounted for
       if (bestDimIdx != -1) {
-        SemiJoinRel semiJoin = possibleDimensions.get(bestDimIdx);
-        SemiJoinRel chosenSemiJoin =
-            new SemiJoinRel(
+        SemiJoin semiJoin = possibleDimensions.get(bestDimIdx);
+        SemiJoin chosenSemiJoin =
+            new SemiJoin(
                 factRel.getCluster(),
                 factRel.getCluster().traitSetOf(Convention.NONE),
                 factRel,
@@ -622,7 +640,7 @@ public class LoptSemiJoinOptimizer {
   private double computeScore(
       RelNode factRel,
       RelNode dimRel,
-      SemiJoinRel semiJoin) {
+      SemiJoin semiJoin) {
     // Estimate savings as a result of applying semijoin filter on fact
     // table.  As a heuristic, the selectivity of the semijoin needs to
     // be less than half.  There may be instances where an even smaller
@@ -695,7 +713,7 @@ public class LoptSemiJoinOptimizer {
    */
   private void removeJoin(
       LoptMultiJoin multiJoin,
-      SemiJoinRel semiJoin,
+      SemiJoin semiJoin,
       int factIdx,
       int dimIdx) {
     // if the dimension can be removed because of another semijoin, then
@@ -774,7 +792,7 @@ public class LoptSemiJoinOptimizer {
    * @param dimIdx index corresponding to dimension table
    */
   private void removePossibleSemiJoin(
-      Map<Integer, SemiJoinRel> possibleDimensions,
+      Map<Integer, SemiJoin> possibleDimensions,
       Integer factIdx,
       Integer dimIdx) {
     // dimension table may not have a corresponding semijoin if it
@@ -824,12 +842,12 @@ public class LoptSemiJoinOptimizer {
   }
 
   /** Dummy class to allow code to compile. */
-  private static class LcsRowScanRel {
+  private static class LcsTableScan {
   }
 
   /** Dummy class to allow code to compile. */
   private static class LcsIndexOptimizer {
-    public LcsIndexOptimizer(LcsRowScanRel rel) {}
+    public LcsIndexOptimizer(LcsTableScan rel) {}
 
     public FemLocalIndex findSemiJoinIndexByCost(RelNode dimRel,
         List<Integer> actualLeftKeys, List<Integer> rightKeys,

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/rules/MultiJoin.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/MultiJoin.java b/core/src/main/java/org/apache/calcite/rel/rules/MultiJoin.java
index 2f252cd..cd70844 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/MultiJoin.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/MultiJoin.java
@@ -14,28 +14,36 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
-
-import java.util.*;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.rex.*;
-import org.eigenbase.util.ImmutableIntList;
-import org.eigenbase.util.ImmutableNullableList;
-
-import net.hydromatic.linq4j.Ord;
+package org.apache.calcite.rel.rules;
+
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.AbstractRelNode;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelWriter;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.util.ImmutableIntList;
+import org.apache.calcite.util.ImmutableNullableList;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Lists;
 
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
 /**
- * A MultiJoinRel represents a join of N inputs, whereas other join relnodes
+ * A MultiJoin represents a join of N inputs, whereas regular Joins
  * represent strictly binary joins.
  */
-public final class MultiJoinRel extends AbstractRelNode {
+public final class MultiJoin extends AbstractRelNode {
   //~ Instance fields --------------------------------------------------------
 
   private final List<RelNode> inputs;
@@ -51,10 +59,10 @@ public final class MultiJoinRel extends AbstractRelNode {
   //~ Constructors -----------------------------------------------------------
 
   /**
-   * Constructs a MultiJoinRel.
+   * Constructs a MultiJoin.
    *
    * @param cluster               cluster that join belongs to
-   * @param inputs                inputs into this multirel join
+   * @param inputs                inputs into this multi-join
    * @param joinFilter            join filter applicable to this join node
    * @param rowType               row type of the join result of this node
    * @param isFullOuterJoin       true if the join is a full outer join
@@ -75,7 +83,7 @@ public final class MultiJoinRel extends AbstractRelNode {
    *                              the input #
    * @param postJoinFilter        filter to be applied after the joins are
    */
-  public MultiJoinRel(
+  public MultiJoin(
       RelOptCluster cluster,
       List<RelNode> inputs,
       RexNode joinFilter,
@@ -108,7 +116,7 @@ public final class MultiJoinRel extends AbstractRelNode {
 
   @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
     assert traitSet.containsIfApplicable(Convention.NONE);
-    return new MultiJoinRel(
+    return new MultiJoin(
         getCluster(),
         inputs,
         joinFilter,
@@ -170,20 +178,19 @@ public final class MultiJoinRel extends AbstractRelNode {
     return inputs;
   }
 
-  @Override
-  public List<RexNode> getChildExps() {
+  @Override public List<RexNode> getChildExps() {
     return ImmutableList.of(joinFilter);
   }
 
   /**
-   * @return join filters associated with this MultiJoinRel
+   * @return join filters associated with this MultiJoin
    */
   public RexNode getJoinFilter() {
     return joinFilter;
   }
 
   /**
-   * @return true if the MultiJoinRel corresponds to a full outer join.
+   * @return true if the MultiJoin corresponds to a full outer join.
    */
   public boolean isFullOuterJoin() {
     return isFullOuterJoin;
@@ -228,7 +235,7 @@ public final class MultiJoinRel extends AbstractRelNode {
   }
 
   /**
-   * @return post-join filter associated with this MultiJoinRel
+   * @return post-join filter associated with this MultiJoin
    */
   public RexNode getPostJoinFilter() {
     return postJoinFilter;
@@ -244,4 +251,4 @@ public final class MultiJoinRel extends AbstractRelNode {
   }
 }
 
-// End MultiJoinRel.java
+// End MultiJoin.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/rules/MultiJoinOptimizeBushyRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/MultiJoinOptimizeBushyRule.java b/core/src/main/java/org/apache/calcite/rel/rules/MultiJoinOptimizeBushyRule.java
index 007e49e..62eef89 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/MultiJoinOptimizeBushyRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/MultiJoinOptimizeBushyRule.java
@@ -14,7 +14,31 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
+package org.apache.calcite.rel.rules;
+
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.prepare.CalcitePrepareImpl;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.core.RelFactories;
+import org.apache.calcite.rel.metadata.RelMdUtil;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexPermuteInputsShuttle;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.rex.RexVisitor;
+import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
+import org.apache.calcite.util.mapping.Mappings;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
 
 import java.io.PrintWriter;
 import java.util.BitSet;
@@ -22,30 +46,15 @@ import java.util.Comparator;
 import java.util.Iterator;
 import java.util.List;
 
-import org.eigenbase.rel.*;
-import org.eigenbase.rel.metadata.RelMdUtil;
-import org.eigenbase.rel.metadata.RelMetadataQuery;
-import org.eigenbase.relopt.*;
-import org.eigenbase.rex.*;
-import org.eigenbase.util.Pair;
-import org.eigenbase.util.Util;
-import org.eigenbase.util.mapping.Mappings;
-
-import net.hydromatic.optiq.prepare.OptiqPrepareImpl;
-import net.hydromatic.optiq.util.BitSets;
-
-import com.google.common.base.Preconditions;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Lists;
-
 /**
  * Planner rule that finds an approximately optimal ordering for join operators
  * using a heuristic algorithm.
  *
- * <p>It is triggered by the pattern {@link ProjectRel} ({@link MultiJoinRel}).
+ * <p>It is triggered by the pattern
+ * {@link org.apache.calcite.rel.logical.LogicalProject} ({@link MultiJoin}).
  *
- * <p>It is similar to {@link org.eigenbase.rel.rules.LoptOptimizeJoinRule}.
+ * <p>It is similar to
+ * {@link org.apache.calcite.rel.rules.LoptOptimizeJoinRule}.
  * {@code LoptOptimizeJoinRule} is only capable of producing left-deep joins;
  * this rule is capable of producing bushy joins.
  *
@@ -57,26 +66,26 @@ import com.google.common.collect.Lists;
  *       e.g. {@code t0.c1 = t1.c1 and t1.c2 = t0.c3}
  * </ol>
  */
-public class OptimizeBushyJoinRule extends RelOptRule {
-  public static final OptimizeBushyJoinRule INSTANCE =
-      new OptimizeBushyJoinRule(RelFactories.DEFAULT_JOIN_FACTORY,
+public class MultiJoinOptimizeBushyRule extends RelOptRule {
+  public static final MultiJoinOptimizeBushyRule INSTANCE =
+      new MultiJoinOptimizeBushyRule(RelFactories.DEFAULT_JOIN_FACTORY,
           RelFactories.DEFAULT_PROJECT_FACTORY);
 
   private final RelFactories.JoinFactory joinFactory;
   private final RelFactories.ProjectFactory projectFactory;
   private final PrintWriter pw =
-      OptiqPrepareImpl.DEBUG ? new PrintWriter(System.out, true) : null;
+      CalcitePrepareImpl.DEBUG ? new PrintWriter(System.out, true) : null;
 
-  /** Creates an OptimizeBushyJoinRule. */
-  public OptimizeBushyJoinRule(RelFactories.JoinFactory joinFactory,
+  /** Creates an MultiJoinOptimizeBushyRule. */
+  public MultiJoinOptimizeBushyRule(RelFactories.JoinFactory joinFactory,
       RelFactories.ProjectFactory projectFactory) {
-    super(operand(MultiJoinRel.class, any()));
+    super(operand(MultiJoin.class, any()));
     this.joinFactory = joinFactory;
     this.projectFactory = projectFactory;
   }
 
   @Override public void onMatch(RelOptRuleCall call) {
-    final MultiJoinRel multiJoinRel = call.rel(0);
+    final MultiJoin multiJoinRel = call.rel(0);
     final RexBuilder rexBuilder = multiJoinRel.getCluster().getRexBuilder();
 
     final LoptMultiJoin multiJoin = new LoptMultiJoin(multiJoinRel);
@@ -369,4 +378,4 @@ public class OptimizeBushyJoinRule extends RelOptRule {
   }
 }
 
-// End OptimizeBushyJoinRule.java
+// End MultiJoinOptimizeBushyRule.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/rules/MultiJoinProjectTransposeRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/MultiJoinProjectTransposeRule.java b/core/src/main/java/org/apache/calcite/rel/rules/MultiJoinProjectTransposeRule.java
index a23f57b..4feb04e 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/MultiJoinProjectTransposeRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/MultiJoinProjectTransposeRule.java
@@ -14,19 +14,28 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
+package org.apache.calcite.rel.rules;
 
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptRuleOperand;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.logical.LogicalJoin;
+import org.apache.calcite.rel.logical.LogicalProject;
 
 /**
- * PullUpProjectsOnTopOfMultiJoinRule implements the rule for pulling {@link
- * ProjectRel}s that are on top of a {@link MultiJoinRel} and beneath a {@link
- * JoinRel} so the {@link ProjectRel} appears above the {@link JoinRel}. In the
- * process of doing so, also save away information about the respective fields
- * that are referenced in the expressions in the {@link ProjectRel} we're
- * pulling up, as well as the join condition, in the resultant {@link
- * MultiJoinRel}s
+ * MultiJoinProjectTransposeRule implements the rule for pulling
+ * {@link org.apache.calcite.rel.logical.LogicalProject}s that are on top of a
+ * {@link MultiJoin} and beneath a
+ * {@link org.apache.calcite.rel.logical.LogicalJoin} so the
+ * {@link org.apache.calcite.rel.logical.LogicalProject} appears above the
+ * {@link org.apache.calcite.rel.logical.LogicalJoin}.
+ *
+ * <p>In the process of doing
+ * so, also save away information about the respective fields that are
+ * referenced in the expressions in the
+ * {@link org.apache.calcite.rel.logical.LogicalProject} we're pulling up, as
+ * well as the join condition, in the resultant {@link MultiJoin}s
  *
  * <p>For example, if we have the following subselect:
  *
@@ -34,49 +43,48 @@ import org.eigenbase.relopt.*;
  *      (select X.x1, Y.y1 from X, Y
  *          where X.x2 = Y.y2 and X.x3 = 1 and Y.y3 = 2)</pre>
  *
- * <p>The {@link MultiJoinRel} associated with (X, Y) associates x1 with X and
+ * <p>The {@link MultiJoin} associated with (X, Y) associates x1 with X and
  * y1 with Y. Although x3 and y3 need to be read due to the filters, they are
  * not required after the row scan has completed and therefore are not saved.
  * The join fields, x2 and y2, are also tracked separately.
  *
- * <p>Note that by only pulling up projects that are on top of {@link
- * MultiJoinRel}s, we preserve projections on top of row scans.
+ * <p>Note that by only pulling up projects that are on top of
+ * {@link MultiJoin}s, we preserve projections on top of row scans.
  *
- * <p>See the superclass for details on restrictions regarding which {@link
- * ProjectRel}s cannot be pulled.
+ * <p>See the superclass for details on restrictions regarding which
+ * {@link org.apache.calcite.rel.logical.LogicalProject}s cannot be pulled.
  */
-public class PullUpProjectsOnTopOfMultiJoinRule
-    extends PullUpProjectsAboveJoinRule {
+public class MultiJoinProjectTransposeRule extends JoinProjectTransposeRule {
   //~ Static fields/initializers ---------------------------------------------
 
-  public static final PullUpProjectsOnTopOfMultiJoinRule MULTI_BOTH_PROJECT =
-      new PullUpProjectsOnTopOfMultiJoinRule(
-          operand(JoinRel.class,
-              operand(ProjectRel.class,
-                  operand(MultiJoinRel.class, any())),
-              operand(ProjectRel.class,
-                  operand(MultiJoinRel.class, any()))),
-          "PullUpProjectsOnTopOfMultiJoinRule: with two ProjectRel children");
+  public static final MultiJoinProjectTransposeRule MULTI_BOTH_PROJECT =
+      new MultiJoinProjectTransposeRule(
+          operand(LogicalJoin.class,
+              operand(LogicalProject.class,
+                  operand(MultiJoin.class, any())),
+              operand(LogicalProject.class,
+                  operand(MultiJoin.class, any()))),
+          "MultiJoinProjectTransposeRule: with two LogicalProject children");
 
-  public static final PullUpProjectsOnTopOfMultiJoinRule MULTI_LEFT_PROJECT =
-      new PullUpProjectsOnTopOfMultiJoinRule(
-          operand(JoinRel.class,
+  public static final MultiJoinProjectTransposeRule MULTI_LEFT_PROJECT =
+      new MultiJoinProjectTransposeRule(
+          operand(LogicalJoin.class,
               some(
-                  operand(ProjectRel.class,
-                      operand(MultiJoinRel.class, any())))),
-          "PullUpProjectsOnTopOfMultiJoinRule: with ProjectRel on left");
+                  operand(LogicalProject.class,
+                      operand(MultiJoin.class, any())))),
+          "MultiJoinProjectTransposeRule: with LogicalProject on left");
 
-  public static final PullUpProjectsOnTopOfMultiJoinRule MULTI_RIGHT_PROJECT =
-      new PullUpProjectsOnTopOfMultiJoinRule(
-          operand(JoinRel.class,
+  public static final MultiJoinProjectTransposeRule MULTI_RIGHT_PROJECT =
+      new MultiJoinProjectTransposeRule(
+          operand(LogicalJoin.class,
               operand(RelNode.class, any()),
-              operand(ProjectRel.class,
-                  operand(MultiJoinRel.class, any()))),
-          "PullUpProjectsOnTopOfMultiJoinRule: with ProjectRel on right");
+              operand(LogicalProject.class,
+                  operand(MultiJoin.class, any()))),
+          "MultiJoinProjectTransposeRule: with LogicalProject on right");
 
   //~ Constructors -----------------------------------------------------------
 
-  public PullUpProjectsOnTopOfMultiJoinRule(
+  public MultiJoinProjectTransposeRule(
       RelOptRuleOperand operand,
       String description) {
     super(operand, description);
@@ -84,18 +92,18 @@ public class PullUpProjectsOnTopOfMultiJoinRule
 
   //~ Methods ----------------------------------------------------------------
 
-  // override PullUpProjectsAboveJoinRule
+  // override JoinProjectTransposeRule
   protected boolean hasLeftChild(RelOptRuleCall call) {
     return call.rels.length != 4;
   }
 
-  // override PullUpProjectsAboveJoinRule
+  // override JoinProjectTransposeRule
   protected boolean hasRightChild(RelOptRuleCall call) {
     return call.rels.length > 3;
   }
 
-  // override PullUpProjectsAboveJoinRule
-  protected ProjectRel getRightChild(RelOptRuleCall call) {
+  // override JoinProjectTransposeRule
+  protected LogicalProject getRightChild(RelOptRuleCall call) {
     if (call.rels.length == 4) {
       return call.rel(2);
     } else {
@@ -103,14 +111,14 @@ public class PullUpProjectsOnTopOfMultiJoinRule
     }
   }
 
-  // override PullUpProjectsAboveJoinRule
+  // override JoinProjectTransposeRule
   protected RelNode getProjectChild(
       RelOptRuleCall call,
-      ProjectRel project,
+      LogicalProject project,
       boolean leftChild) {
-    // locate the appropriate MultiJoinRel based on which rule was fired
+    // locate the appropriate MultiJoin based on which rule was fired
     // and which projection we're dealing with
-    MultiJoinRel multiJoin;
+    MultiJoin multiJoin;
     if (leftChild) {
       multiJoin = call.rel(2);
     } else if (call.rels.length == 4) {
@@ -119,10 +127,10 @@ public class PullUpProjectsOnTopOfMultiJoinRule
       multiJoin = call.rel(4);
     }
 
-    // create a new MultiJoinRel that reflects the columns in the projection
-    // above the MultiJoinRel
+    // create a new MultiJoin that reflects the columns in the projection
+    // above the MultiJoin
     return RelOptUtil.projectMultiJoin(multiJoin, project);
   }
 }
 
-// End PullUpProjectsOnTopOfMultiJoinRule.java
+// End MultiJoinProjectTransposeRule.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/rules/ProjectCalcMergeRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/ProjectCalcMergeRule.java b/core/src/main/java/org/apache/calcite/rel/rules/ProjectCalcMergeRule.java
index 999b7fa..40f4aab 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/ProjectCalcMergeRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/ProjectCalcMergeRule.java
@@ -14,42 +14,55 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
+package org.apache.calcite.rel.rules;
 
-import java.util.*;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.logical.LogicalCalc;
+import org.apache.calcite.rel.logical.LogicalProject;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexOver;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexProgramBuilder;
+import org.apache.calcite.util.Pair;
 
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.rex.*;
-import org.eigenbase.util.Pair;
+import java.util.Collections;
 
 /**
- * Planner rule which merges a {@link ProjectRel} and a {@link CalcRel}. The
- * resulting {@link CalcRel} has the same project list as the original {@link
- * ProjectRel}, but expressed in terms of the original {@link CalcRel}'s inputs.
+ * Planner rule which merges a
+ * {@link org.apache.calcite.rel.logical.LogicalProject} and a
+ * {@link org.apache.calcite.rel.logical.LogicalCalc}.
  *
- * @see MergeFilterOntoCalcRule
+ * <p>The resulting {@link org.apache.calcite.rel.logical.LogicalCalc} has the
+ * same project list as the original
+ * {@link org.apache.calcite.rel.logical.LogicalProject}, but expressed in terms
+ * of the original {@link org.apache.calcite.rel.logical.LogicalCalc}'s inputs.
+ *
+ * @see FilterCalcMergeRule
  */
-public class MergeProjectOntoCalcRule extends RelOptRule {
+public class ProjectCalcMergeRule extends RelOptRule {
   //~ Static fields/initializers ---------------------------------------------
 
-  public static final MergeProjectOntoCalcRule INSTANCE =
-      new MergeProjectOntoCalcRule();
+  public static final ProjectCalcMergeRule INSTANCE =
+      new ProjectCalcMergeRule();
 
   //~ Constructors -----------------------------------------------------------
 
-  private MergeProjectOntoCalcRule() {
+  private ProjectCalcMergeRule() {
     super(
         operand(
-            ProjectRel.class,
-            operand(CalcRel.class, any())));
+            LogicalProject.class,
+            operand(LogicalCalc.class, any())));
   }
 
   //~ Methods ----------------------------------------------------------------
 
   public void onMatch(RelOptRuleCall call) {
-    final ProjectRel project = call.rel(0);
-    final CalcRel calc = call.rel(1);
+    final LogicalProject project = call.rel(0);
+    final LogicalCalc calc = call.rel(1);
 
     // Don't merge a project which contains windowed aggregates onto a
     // calc. That would effectively be pushing a windowed aggregate down
@@ -65,8 +78,8 @@ public class MergeProjectOntoCalcRule extends RelOptRule {
             project.getRowType(),
             cluster.getRexBuilder());
     if (RexOver.containsOver(program)) {
-      CalcRel projectAsCalc =
-          new CalcRel(
+      LogicalCalc projectAsCalc =
+          new LogicalCalc(
               cluster,
               project.getTraitSet(),
               calc,
@@ -95,11 +108,11 @@ public class MergeProjectOntoCalcRule extends RelOptRule {
             topProgram,
             bottomProgram,
             rexBuilder);
-    final CalcRel newCalc =
-        new CalcRel(
+    final LogicalCalc newCalc =
+        new LogicalCalc(
             cluster,
             project.getTraitSet(),
-            calc.getChild(),
+            calc.getInput(),
             project.getRowType(),
             mergedProgram,
             Collections.<RelCollation>emptyList());
@@ -107,4 +120,4 @@ public class MergeProjectOntoCalcRule extends RelOptRule {
   }
 }
 
-// End MergeProjectOntoCalcRule.java
+// End ProjectCalcMergeRule.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/rules/ProjectFilterTransposeRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/ProjectFilterTransposeRule.java b/core/src/main/java/org/apache/calcite/rel/rules/ProjectFilterTransposeRule.java
index 70ba40e..0fb7918 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/ProjectFilterTransposeRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/ProjectFilterTransposeRule.java
@@ -14,19 +14,23 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
+package org.apache.calcite.rel.rules;
 
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.rex.*;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.logical.LogicalFilter;
+import org.apache.calcite.rel.logical.LogicalProject;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexOver;
 
 /**
- * PushProjectPastFilterRule implements the rule for pushing a projection past a
- * filter.
+ * Planner rule that pushes a {@link org.apache.calcite.rel.core.Project}
+ * past a {@link org.apache.calcite.rel.core.Filter}.
  */
-public class PushProjectPastFilterRule extends RelOptRule {
-  public static final PushProjectPastFilterRule INSTANCE =
-      new PushProjectPastFilterRule(PushProjector.ExprCondition.FALSE);
+public class ProjectFilterTransposeRule extends RelOptRule {
+  public static final ProjectFilterTransposeRule INSTANCE =
+      new ProjectFilterTransposeRule(PushProjector.ExprCondition.FALSE);
 
   //~ Instance fields --------------------------------------------------------
 
@@ -38,17 +42,17 @@ public class PushProjectPastFilterRule extends RelOptRule {
   //~ Constructors -----------------------------------------------------------
 
   /**
-   * Creates a PushProjectPastFilterRule.
+   * Creates a ProjectFilterTransposeRule.
    *
    * @param preserveExprCondition Condition for expressions that should be
    *                              preserved in the projection
    */
-  private PushProjectPastFilterRule(
+  private ProjectFilterTransposeRule(
       PushProjector.ExprCondition preserveExprCondition) {
     super(
         operand(
-            ProjectRel.class,
-            operand(FilterRel.class, any())));
+            LogicalProject.class,
+            operand(LogicalFilter.class, any())));
     this.preserveExprCondition = preserveExprCondition;
   }
 
@@ -56,18 +60,18 @@ public class PushProjectPastFilterRule extends RelOptRule {
 
   // implement RelOptRule
   public void onMatch(RelOptRuleCall call) {
-    ProjectRel origProj;
-    FilterRel filterRel;
+    LogicalProject origProj;
+    LogicalFilter filter;
 
     if (call.rels.length == 2) {
       origProj = call.rel(0);
-      filterRel = call.rel(1);
+      filter = call.rel(1);
     } else {
       origProj = null;
-      filterRel = call.rel(0);
+      filter = call.rel(0);
     }
-    RelNode rel = filterRel.getChild();
-    RexNode origFilter = filterRel.getCondition();
+    RelNode rel = filter.getInput();
+    RexNode origFilter = filter.getCondition();
 
     if ((origProj != null)
         && RexOver.containsOver(origProj.getProjects(), null)) {
@@ -90,4 +94,4 @@ public class PushProjectPastFilterRule extends RelOptRule {
   }
 }
 
-// End PushProjectPastFilterRule.java
+// End ProjectFilterTransposeRule.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/rules/ProjectJoinTransposeRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/ProjectJoinTransposeRule.java b/core/src/main/java/org/apache/calcite/rel/rules/ProjectJoinTransposeRule.java
index 9d6eddb..5379110 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/ProjectJoinTransposeRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/ProjectJoinTransposeRule.java
@@ -14,23 +14,28 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
+package org.apache.calcite.rel.rules;
 
-import java.util.*;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.logical.LogicalProject;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexNode;
 
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.rex.*;
+import java.util.ArrayList;
+import java.util.List;
 
 /**
- * PushProjectPastJoinRule implements the rule for pushing a projection past a
- * join by splitting the projection into a projection on top of each child of
+ * Planner rule that pushes a {@link org.apache.calcite.rel.core.Project}
+ * past a {@link org.apache.calcite.rel.core.Join}
+ * by splitting the projection into a projection on top of each child of
  * the join.
  */
-public class PushProjectPastJoinRule extends RelOptRule {
-  public static final PushProjectPastJoinRule INSTANCE =
-      new PushProjectPastJoinRule(PushProjector.ExprCondition.FALSE);
+public class ProjectJoinTransposeRule extends RelOptRule {
+  public static final ProjectJoinTransposeRule INSTANCE =
+      new ProjectJoinTransposeRule(PushProjector.ExprCondition.FALSE);
 
   //~ Instance fields --------------------------------------------------------
 
@@ -42,16 +47,16 @@ public class PushProjectPastJoinRule extends RelOptRule {
   //~ Constructors -----------------------------------------------------------
 
   /**
-   * Creates a PushProjectPastJoinRule with an explicit condition.
+   * Creates a ProjectJoinTransposeRule with an explicit condition.
    *
    * @param preserveExprCondition Condition for expressions that should be
    *                              preserved in the projection
    */
-  private PushProjectPastJoinRule(
+  private ProjectJoinTransposeRule(
       PushProjector.ExprCondition preserveExprCondition) {
     super(
-        operand(ProjectRel.class,
-            operand(JoinRelBase.class, any())));
+        operand(LogicalProject.class,
+            operand(Join.class, any())));
     this.preserveExprCondition = preserveExprCondition;
   }
 
@@ -59,8 +64,8 @@ public class PushProjectPastJoinRule extends RelOptRule {
 
   // implement RelOptRule
   public void onMatch(RelOptRuleCall call) {
-    ProjectRel origProj = call.rel(0);
-    final JoinRelBase join = call.rel(1);
+    LogicalProject origProj = call.rel(0);
+    final Join join = call.rel(1);
 
     // locate all fields referenced in the projection and join condition;
     // determine which inputs are referenced in the projection and
@@ -109,7 +114,7 @@ public class PushProjectPastJoinRule extends RelOptRule {
     }
 
     // create a new join with the projected children
-    JoinRelBase newJoinRel =
+    Join newJoinRel =
         join.copy(
             join.getTraitSet(),
             newJoinFilter,
@@ -127,4 +132,4 @@ public class PushProjectPastJoinRule extends RelOptRule {
   }
 }
 
-// End PushProjectPastJoinRule.java
+// End ProjectJoinTransposeRule.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/rules/ProjectMergeRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/ProjectMergeRule.java b/core/src/main/java/org/apache/calcite/rel/rules/ProjectMergeRule.java
index 78c67af..96559cf 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/ProjectMergeRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/ProjectMergeRule.java
@@ -14,30 +14,36 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
-
-import java.util.*;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.rel.RelFactories.ProjectFactory;
-import org.eigenbase.relopt.*;
-import org.eigenbase.rex.*;
-import org.eigenbase.util.Permutation;
+package org.apache.calcite.rel.rules;
+
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.RelFactories;
+import org.apache.calcite.rel.core.RelFactories.ProjectFactory;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexLocalRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexProgramBuilder;
+import org.apache.calcite.util.Permutation;
+
+import java.util.ArrayList;
+import java.util.List;
 
 /**
- * MergeProjectRule merges a {@link ProjectRelBase} into
- * another {@link ProjectRelBase},
+ * ProjectMergeRule merges a {@link org.apache.calcite.rel.core.Project} into
+ * another {@link org.apache.calcite.rel.core.Project},
  * provided the projects aren't projecting identical sets of input references.
  */
-public class MergeProjectRule extends RelOptRule {
-  public static final MergeProjectRule INSTANCE =
-      new MergeProjectRule();
+public class ProjectMergeRule extends RelOptRule {
+  public static final ProjectMergeRule INSTANCE = new ProjectMergeRule();
 
   //~ Instance fields --------------------------------------------------------
 
-  /**
-   * if true, always merge projects
-   */
+  /** Whether to always merge projects. */
   private final boolean force;
 
   private final ProjectFactory projectFactory;
@@ -45,22 +51,22 @@ public class MergeProjectRule extends RelOptRule {
   //~ Constructors -----------------------------------------------------------
 
   /**
-   * Creates a MergeProjectRule.
+   * Creates a ProjectMergeRule.
    */
-  private MergeProjectRule() {
+  private ProjectMergeRule() {
     this(false, RelFactories.DEFAULT_PROJECT_FACTORY);
   }
 
   /**
-   * Creates a MergeProjectRule, specifying whether to always merge projects.
+   * Creates a ProjectMergeRule, specifying whether to always merge projects.
    *
    * @param force Whether to always merge projects
    */
-  public MergeProjectRule(boolean force, ProjectFactory pFactory) {
+  public ProjectMergeRule(boolean force, ProjectFactory pFactory) {
     super(
-        operand(ProjectRelBase.class,
-            operand(ProjectRelBase.class, any())),
-        "MergeProjectRule" + (force ? ": force mode" : ""));
+        operand(Project.class,
+            operand(Project.class, any())),
+        "ProjectMergeRule" + (force ? ": force mode" : ""));
     this.force = force;
     projectFactory = pFactory;
   }
@@ -69,8 +75,8 @@ public class MergeProjectRule extends RelOptRule {
 
   // implement RelOptRule
   public void onMatch(RelOptRuleCall call) {
-    ProjectRelBase topProject = call.rel(0);
-    ProjectRelBase bottomProject = call.rel(1);
+    Project topProject = call.rel(0);
+    Project bottomProject = call.rel(1);
     RexBuilder rexBuilder = topProject.getCluster().getRexBuilder();
 
     // If one or both projects are permutations, short-circuit the complex logic
@@ -78,18 +84,18 @@ public class MergeProjectRule extends RelOptRule {
     final Permutation topPermutation = topProject.getPermutation();
     if (topPermutation != null) {
       if (topPermutation.isIdentity()) {
-        // Let RemoveTrivialProjectRule handle this.
+        // Let ProjectRemoveRule handle this.
         return;
       }
       final Permutation bottomPermutation = bottomProject.getPermutation();
       if (bottomPermutation != null) {
         if (bottomPermutation.isIdentity()) {
-          // Let RemoveTrivialProjectRule handle this.
+          // Let ProjectRemoveRule handle this.
           return;
         }
         final Permutation product = topPermutation.product(bottomPermutation);
         call.transformTo(
-            RelOptUtil.projectMapping(bottomProject.getChild(),
+            RelOptUtil.projectMapping(bottomProject.getInput(),
                 product.inverse(), topProject.getRowType().getFieldNames(),
                 projectFactory));
         return;
@@ -98,7 +104,7 @@ public class MergeProjectRule extends RelOptRule {
 
     // if we're not in force mode and the two projects reference identical
     // inputs, then return and either let FennelRenameRule or
-    // RemoveTrivialProjectRule replace the projects
+    // ProjectRemoveRule replace the projects
     if (!force) {
       if (RelOptUtil.checkProjAndChildInputs(topProject, false)) {
         return;
@@ -108,7 +114,7 @@ public class MergeProjectRule extends RelOptRule {
     // create a RexProgram for the bottom project
     RexProgram bottomProgram =
         RexProgram.create(
-            bottomProject.getChild().getRowType(),
+            bottomProject.getInput().getRowType(),
             bottomProject.getProjects(),
             null,
             bottomProject.getRowType(),
@@ -142,11 +148,11 @@ public class MergeProjectRule extends RelOptRule {
 
     // replace the two projects with a combined projection
     RelNode newProjectRel = projectFactory.createProject(
-        bottomProject.getChild(), newProjExprs,
+        bottomProject.getInput(), newProjExprs,
         topProject.getRowType().getFieldNames());
 
     call.transformTo(newProjectRel);
   }
 }
 
-// End MergeProjectRule.java
+// End ProjectMergeRule.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/rules/ProjectMultiJoinMergeRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/ProjectMultiJoinMergeRule.java b/core/src/main/java/org/apache/calcite/rel/rules/ProjectMultiJoinMergeRule.java
index fd300c9..065738e 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/ProjectMultiJoinMergeRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/ProjectMultiJoinMergeRule.java
@@ -14,37 +14,42 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
+package org.apache.calcite.rel.rules;
 
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.logical.LogicalProject;
 
 /**
- * PushProjectIntoMultiJoinRule implements the rule for pushing projection
- * information from a {@link ProjectRel} into the {@link MultiJoinRel} that is
- * input into the {@link ProjectRel}.
+ * Planner rule that pushes
+ * {@link org.apache.calcite.rel.core.Project}
+ * into a {@link MultiJoin},
+ * creating a richer {@code MultiJoin}.
+ *
+ * @see org.apache.calcite.rel.rules.FilterMultiJoinMergeRule
  */
-public class PushProjectIntoMultiJoinRule extends RelOptRule {
-  public static final PushProjectIntoMultiJoinRule INSTANCE =
-      new PushProjectIntoMultiJoinRule();
+public class ProjectMultiJoinMergeRule extends RelOptRule {
+  public static final ProjectMultiJoinMergeRule INSTANCE =
+      new ProjectMultiJoinMergeRule();
 
   //~ Constructors -----------------------------------------------------------
 
   /**
-   * Creates a PushProjectIntoMultiJoinRule.
+   * Creates a ProjectMultiJoinMergeRule.
    */
-  private PushProjectIntoMultiJoinRule() {
+  private ProjectMultiJoinMergeRule() {
     super(
         operand(
-            ProjectRel.class,
-            operand(MultiJoinRel.class, any())));
+            LogicalProject.class,
+            operand(MultiJoin.class, any())));
   }
 
   //~ Methods ----------------------------------------------------------------
 
   public void onMatch(RelOptRuleCall call) {
-    ProjectRel project = call.rel(0);
-    MultiJoinRel multiJoin = call.rel(1);
+    LogicalProject project = call.rel(0);
+    MultiJoin multiJoin = call.rel(1);
 
     // if all inputs have their projFields set, then projection information
     // has already been pushed into each input
@@ -59,12 +64,12 @@ public class PushProjectIntoMultiJoinRule extends RelOptRule {
       return;
     }
 
-    // create a new MultiJoinRel that reflects the columns in the projection
-    // above the MultiJoinRel
-    MultiJoinRel newMultiJoin =
+    // create a new MultiJoin that reflects the columns in the projection
+    // above the MultiJoin
+    MultiJoin newMultiJoin =
         RelOptUtil.projectMultiJoin(multiJoin, project);
-    ProjectRel newProject =
-        (ProjectRel) RelOptUtil.createProject(
+    LogicalProject newProject =
+        (LogicalProject) RelOptUtil.createProject(
             newMultiJoin,
             project.getProjects(),
             project.getRowType().getFieldNames());
@@ -73,4 +78,4 @@ public class PushProjectIntoMultiJoinRule extends RelOptRule {
   }
 }
 
-// End PushProjectIntoMultiJoinRule.java
+// End ProjectMultiJoinMergeRule.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/rules/ProjectRemoveRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/ProjectRemoveRule.java b/core/src/main/java/org/apache/calcite/rel/rules/ProjectRemoveRule.java
index c84098b..b74f532 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/ProjectRemoveRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/ProjectRemoveRule.java
@@ -14,54 +14,58 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
+package org.apache.calcite.rel.rules;
 
-import java.util.List;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.rex.*;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexNode;
 
 import com.google.common.base.Predicate;
 
+import java.util.List;
+
 /**
- * Rule that, given a {@link ProjectRelBase} node that merely returns its input,
- * converts the node into its child.
+ * Planner rule that,
+ * given a {@link org.apache.calcite.rel.core.Project} node that
+ * merely returns its input, converts the node into its child.
  *
- * <p>For example, <code>ProjectRel(ArrayReader(a), {$input0})</code> becomes
+ * <p>For example, <code>Project(ArrayReader(a), {$input0})</code> becomes
  * <code>ArrayReader(a)</code>.</p>
  *
- * @see org.eigenbase.rel.rules.RemoveTrivialCalcRule
- * @see org.eigenbase.rel.rules.MergeProjectRule
+ * @see CalcRemoveRule
+ * @see ProjectMergeRule
  */
-public class RemoveTrivialProjectRule extends RelOptRule {
+public class ProjectRemoveRule extends RelOptRule {
   //~ Static fields/initializers ---------------------------------------------
 
-  private static final Predicate<ProjectRelBase> PREDICATE =
-      new Predicate<ProjectRelBase>() {
-        public boolean apply(ProjectRelBase input) {
+  private static final Predicate<Project> PREDICATE =
+      new Predicate<Project>() {
+        public boolean apply(Project input) {
           return isTrivial(input);
         }
       };
 
-  public static final RemoveTrivialProjectRule INSTANCE =
-      new RemoveTrivialProjectRule();
+  public static final ProjectRemoveRule INSTANCE = new ProjectRemoveRule();
 
   //~ Constructors -----------------------------------------------------------
 
-  private RemoveTrivialProjectRule() {
+  private ProjectRemoveRule() {
     // Create a specialized operand to detect non-matches early. This keeps
     // the rule queue short.
-    super(operand(ProjectRelBase.class, null, PREDICATE, any()));
+    super(operand(Project.class, null, PREDICATE, any()));
   }
 
   //~ Methods ----------------------------------------------------------------
 
   public void onMatch(RelOptRuleCall call) {
-    ProjectRelBase project = call.rel(0);
+    Project project = call.rel(0);
     assert isTrivial(project);
-    RelNode stripped = project.getChild();
+    RelNode stripped = project.getInput();
     RelNode child = call.getPlanner().register(stripped, project);
     call.transformTo(child);
   }
@@ -70,12 +74,12 @@ public class RemoveTrivialProjectRule extends RelOptRule {
    * Returns the child of a project if the project is trivial, otherwise
    * the project itself.
    */
-  public static RelNode strip(ProjectRelBase project) {
-    return isTrivial(project) ? project.getChild() : project;
+  public static RelNode strip(Project project) {
+    return isTrivial(project) ? project.getInput() : project;
   }
 
-  public static boolean isTrivial(ProjectRelBase project) {
-    RelNode child = project.getChild();
+  public static boolean isTrivial(Project project) {
+    RelNode child = project.getInput();
     final RelDataType childRowType = child.getRowType();
     if (!childRowType.isStruct()) {
       return false;
@@ -83,19 +87,15 @@ public class RemoveTrivialProjectRule extends RelOptRule {
     if (!project.isBoxed()) {
       return false;
     }
-    if (!isIdentity(
-        project.getProjects(),
-        project.getRowType(),
+    if (!isIdentity(project.getProjects(), project.getRowType(),
         childRowType)) {
       return false;
     }
     return true;
   }
 
-  public static boolean isIdentity(
-      List<? extends RexNode> exps,
-      RelDataType rowType,
-      RelDataType childRowType) {
+  public static boolean isIdentity(List<? extends RexNode> exps,
+      RelDataType rowType, RelDataType childRowType) {
     List<RelDataTypeField> fields = rowType.getFieldList();
     List<RelDataTypeField> childFields = childRowType.getFieldList();
     int fieldCount = childFields.size();
@@ -119,4 +119,4 @@ public class RemoveTrivialProjectRule extends RelOptRule {
   }
 }
 
-// End RemoveTrivialProjectRule.java
+// End ProjectRemoveRule.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/rules/ProjectSetOpTransposeRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/ProjectSetOpTransposeRule.java b/core/src/main/java/org/apache/calcite/rel/rules/ProjectSetOpTransposeRule.java
index 845deee..2647c5c 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/ProjectSetOpTransposeRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/ProjectSetOpTransposeRule.java
@@ -14,22 +14,30 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
+package org.apache.calcite.rel.rules;
 
-import java.util.*;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.SetOp;
+import org.apache.calcite.rel.logical.LogicalProject;
+import org.apache.calcite.rex.RexInputRef;
 
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.rex.*;
+import java.util.ArrayList;
+import java.util.List;
 
 /**
- * PushProjectPastSetOpRule implements the rule for pushing a {@link ProjectRel}
- * past a {@link SetOpRel}. The children of the {@link SetOpRel} will project
- * only the {@link RexInputRef}s referenced in the original {@link ProjectRel}.
+ * Planner rule that pushes
+ * a {@link org.apache.calcite.rel.logical.LogicalProject}
+ * past a {@link org.apache.calcite.rel.core.SetOp}.
+ *
+ * <p>The children of the {@code SetOp} will project
+ * only the {@link RexInputRef}s referenced in the original
+ * {@code LogicalProject}.
  */
-public class PushProjectPastSetOpRule extends RelOptRule {
-  public static final PushProjectPastSetOpRule INSTANCE =
-      new PushProjectPastSetOpRule(PushProjector.ExprCondition.FALSE);
+public class ProjectSetOpTransposeRule extends RelOptRule {
+  public static final ProjectSetOpTransposeRule INSTANCE =
+      new ProjectSetOpTransposeRule(PushProjector.ExprCondition.FALSE);
 
   //~ Instance fields --------------------------------------------------------
 
@@ -41,17 +49,17 @@ public class PushProjectPastSetOpRule extends RelOptRule {
   //~ Constructors -----------------------------------------------------------
 
   /**
-   * Creates a PushProjectPastSetOpRule with an explicit condition whether
+   * Creates a ProjectSetOpTransposeRule with an explicit condition whether
    * to preserve expressions.
    *
    * @param preserveExprCondition Condition whether to preserve expressions
    */
-  public PushProjectPastSetOpRule(
+  public ProjectSetOpTransposeRule(
       PushProjector.ExprCondition preserveExprCondition) {
     super(
         operand(
-            ProjectRel.class,
-            operand(SetOpRel.class, any())));
+            LogicalProject.class,
+            operand(SetOp.class, any())));
     this.preserveExprCondition = preserveExprCondition;
   }
 
@@ -59,17 +67,17 @@ public class PushProjectPastSetOpRule extends RelOptRule {
 
   // implement RelOptRule
   public void onMatch(RelOptRuleCall call) {
-    ProjectRel origProj = call.rel(0);
-    SetOpRel setOpRel = call.rel(1);
+    LogicalProject origProj = call.rel(0);
+    SetOp setOp = call.rel(1);
 
     // cannot push project past a distinct
-    if (!setOpRel.all) {
+    if (!setOp.all) {
       return;
     }
 
     // locate all fields referenced in the projection
     PushProjector pushProject =
-        new PushProjector(origProj, null, setOpRel, preserveExprCondition);
+        new PushProjector(origProj, null, setOp, preserveExprCondition);
     pushProject.locateAllRefs();
 
     List<RelNode> newSetOpInputs = new ArrayList<RelNode>();
@@ -80,10 +88,10 @@ public class PushProjectPastSetOpRule extends RelOptRule {
     // to try to keep expensive expressions above the join,
     // because UNION ALL does not have any filtering effect,
     // and it is the only operator this rule currently acts on
-    for (RelNode input : setOpRel.getInputs()) {
+    for (RelNode input : setOp.getInputs()) {
       // be lazy:  produce two ProjectRels, and let another rule
       // merge them (could probably just clone origProj instead?)
-      ProjectRel p =
+      LogicalProject p =
           pushProject.createProjectRefsAndExprs(
               input, true, false);
       newSetOpInputs.add(
@@ -91,11 +99,11 @@ public class PushProjectPastSetOpRule extends RelOptRule {
     }
 
     // create a new setop whose children are the ProjectRels created above
-    SetOpRel newSetOpRel =
-        setOpRel.copy(setOpRel.getTraitSet(), newSetOpInputs);
+    SetOp newSetOp =
+        setOp.copy(setOp.getTraitSet(), newSetOpInputs);
 
-    call.transformTo(newSetOpRel);
+    call.transformTo(newSetOp);
   }
 }
 
-// End PushProjectPastSetOpRule.java
+// End ProjectSetOpTransposeRule.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/rules/ProjectSortTransposeRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/ProjectSortTransposeRule.java b/core/src/main/java/org/apache/calcite/rel/rules/ProjectSortTransposeRule.java
index 6803e3c..fe53113 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/ProjectSortTransposeRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/ProjectSortTransposeRule.java
@@ -14,45 +14,50 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
+package org.apache.calcite.rel.rules;
 
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.logical.LogicalProject;
 
 import com.google.common.collect.ImmutableList;
 
 /**
- * Planner rule that pushes a {@link ProjectRel} past a {@link SortRel}.
+ * Planner rule that pushes
+ * a {@link org.apache.calcite.rel.logical.LogicalProject}
+ * past a {@link org.apache.calcite.rel.core.Sort}.
+ *
+ * @see org.apache.calcite.rel.rules.SortProjectTransposeRule
  */
-public class PushProjectPastSortRule extends RelOptRule {
-  public static final PushProjectPastSortRule INSTANCE =
-      new PushProjectPastSortRule();
+public class ProjectSortTransposeRule extends RelOptRule {
+  public static final ProjectSortTransposeRule INSTANCE =
+      new ProjectSortTransposeRule();
 
   //~ Constructors -----------------------------------------------------------
 
   /**
-   * Creates a PushProjectPastSortRule.
+   * Creates a ProjectSortTransposeRule.
    */
-  private PushProjectPastSortRule() {
+  private ProjectSortTransposeRule() {
     super(
-        operand(
-            ProjectRel.class,
-            operand(SortRel.class, any())));
+        operand(LogicalProject.class,
+            operand(Sort.class, any())));
   }
 
   //~ Methods ----------------------------------------------------------------
 
-  // implement RelOptRule
   public void onMatch(RelOptRuleCall call) {
-    ProjectRel project = call.rel(0);
-    SortRel sort = call.rel(1);
-    if (sort.getClass() != SortRel.class) {
+    LogicalProject project = call.rel(0);
+    Sort sort = call.rel(1);
+    if (sort.getClass() != Sort.class) {
       return;
     }
     RelNode newProject =
         project.copy(
-            project.getTraitSet(), ImmutableList.of(sort.getChild()));
-    final SortRel newSort =
+            project.getTraitSet(), ImmutableList.of(sort.getInput()));
+    final Sort newSort =
         sort.copy(
             sort.getTraitSet(),
             newProject,
@@ -63,4 +68,4 @@ public class PushProjectPastSortRule extends RelOptRule {
   }
 }
 
-// End PushProjectPastSortRule.java
+// End ProjectSortTransposeRule.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/rules/ProjectTableRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/ProjectTableRule.java b/core/src/main/java/org/apache/calcite/rel/rules/ProjectTableRule.java
index 475edbb..785cc26 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/ProjectTableRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/ProjectTableRule.java
@@ -14,43 +14,42 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
+package org.apache.calcite.rel.rules;
 
-import java.util.List;
-
-import org.eigenbase.rel.FilterRelBase;
-import org.eigenbase.rel.ProjectRelBase;
-import org.eigenbase.rel.RelNode;
-import org.eigenbase.rel.TableAccessRelBase;
-import org.eigenbase.relopt.RelOptRule;
-import org.eigenbase.relopt.RelOptRuleCall;
-import org.eigenbase.relopt.RelOptRuleOperand;
-import org.eigenbase.relopt.RelOptTable;
-import org.eigenbase.relopt.RelOptUtil;
-import org.eigenbase.rex.RexBuilder;
-import org.eigenbase.rex.RexInputRef;
-import org.eigenbase.rex.RexLocalRef;
-import org.eigenbase.rex.RexNode;
-import org.eigenbase.rex.RexProgram;
-import org.eigenbase.rex.RexShuttle;
-
-import net.hydromatic.optiq.ProjectableFilterableTable;
-import net.hydromatic.optiq.rules.java.EnumerableRel;
-import net.hydromatic.optiq.rules.java.JavaRules;
+import org.apache.calcite.adapter.enumerable.EnumerableInterpreter;
+import org.apache.calcite.adapter.enumerable.EnumerableRel;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptRuleOperand;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLocalRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexShuttle;
+import org.apache.calcite.schema.ProjectableFilterableTable;
 
 import com.google.common.base.Predicate;
 import com.google.common.collect.Lists;
 
+import java.util.List;
+
 /**
  * Planner rule that pushes a project into a scan of a
- * {@link net.hydromatic.optiq.ProjectableFilterableTable}.
+ * {@link org.apache.calcite.schema.ProjectableFilterableTable}.
  *
- * @see org.eigenbase.rel.rules.FilterTableRule
+ * @see org.apache.calcite.rel.rules.FilterTableRule
  */
 public abstract class ProjectTableRule extends RelOptRule {
-  private static final Predicate<TableAccessRelBase> PREDICATE =
-      new Predicate<TableAccessRelBase>() {
-        public boolean apply(TableAccessRelBase scan) {
+  private static final Predicate<TableScan> PREDICATE =
+      new Predicate<TableScan>() {
+        public boolean apply(TableScan scan) {
           // We can only push projects into a ProjectableFilterableTable.
           final RelOptTable table = scan.getTable();
           return table.unwrap(ProjectableFilterableTable.class) != null;
@@ -59,14 +58,14 @@ public abstract class ProjectTableRule extends RelOptRule {
 
   public static final ProjectTableRule INSTANCE =
       new ProjectTableRule(
-          operand(ProjectRelBase.class,
-              operand(JavaRules.EnumerableInterpreterRel.class,
-                  operand(TableAccessRelBase.class, null, PREDICATE, none()))),
+          operand(Project.class,
+              operand(EnumerableInterpreter.class,
+                  operand(TableScan.class, null, PREDICATE, none()))),
           "ProjectTableRule:basic") {
         @Override public void onMatch(RelOptRuleCall call) {
-          final ProjectRelBase project = call.rel(0);
-          final JavaRules.EnumerableInterpreterRel interpreter = call.rel(1);
-          final TableAccessRelBase scan = call.rel(2);
+          final Project project = call.rel(0);
+          final EnumerableInterpreter interpreter = call.rel(1);
+          final TableScan scan = call.rel(2);
           final RelOptTable table = scan.getTable();
           assert table.unwrap(ProjectableFilterableTable.class) != null;
           apply(call, project, null, interpreter);
@@ -75,17 +74,17 @@ public abstract class ProjectTableRule extends RelOptRule {
 
   public static final ProjectTableRule INSTANCE2 =
       new ProjectTableRule(
-          operand(ProjectRelBase.class,
-              operand(FilterRelBase.class,
-                  operand(JavaRules.EnumerableInterpreterRel.class,
-                      operand(TableAccessRelBase.class, null, PREDICATE,
+          operand(Project.class,
+              operand(Filter.class,
+                  operand(EnumerableInterpreter.class,
+                      operand(TableScan.class, null, PREDICATE,
                           none())))),
           "ProjectTableRule:filter") {
         @Override public void onMatch(RelOptRuleCall call) {
-          final ProjectRelBase project = call.rel(0);
-          final FilterRelBase filter = call.rel(1);
-          final JavaRules.EnumerableInterpreterRel interpreter = call.rel(2);
-          final TableAccessRelBase scan = call.rel(3);
+          final Project project = call.rel(0);
+          final Filter filter = call.rel(1);
+          final EnumerableInterpreter interpreter = call.rel(2);
+          final TableScan scan = call.rel(3);
           final RelOptTable table = scan.getTable();
           assert table.unwrap(ProjectableFilterableTable.class) != null;
           apply(call, project, filter, interpreter);
@@ -101,8 +100,8 @@ public abstract class ProjectTableRule extends RelOptRule {
 
   //~ Methods ----------------------------------------------------------------
 
-  protected void apply(RelOptRuleCall call, ProjectRelBase project,
-      FilterRelBase filter, JavaRules.EnumerableInterpreterRel interpreter) {
+  protected void apply(RelOptRuleCall call, Project project,
+      Filter filter, EnumerableInterpreter interpreter) {
     // Split the projects into column references and expressions on top of them.
     // Creating a RexProgram is a convenient way to do this.
     final RexBuilder rexBuilder = project.getCluster().getRexBuilder();
@@ -142,7 +141,7 @@ public abstract class ProjectTableRule extends RelOptRule {
       }
     }
 
-    RelNode input = interpreter.getChild();
+    RelNode input = interpreter.getInput();
     if (filter != null) {
       input = RelOptUtil.createFilter(input, filter.getCondition(),
           EnumerableRel.FILTER_FACTORY);
@@ -151,7 +150,7 @@ public abstract class ProjectTableRule extends RelOptRule {
         RelOptUtil.createProject(EnumerableRel.PROJECT_FACTORY, input,
             projectOrdinals);
     final RelNode newInterpreter =
-        new JavaRules.EnumerableInterpreterRel(interpreter.getCluster(),
+        new EnumerableInterpreter(interpreter.getCluster(),
             interpreter.getTraitSet(), newProject, 0.15d);
     final RelNode residue;
     if (extraProjects != null) {


[35/58] [abbrv] [partial] incubator-calcite git commit: [CALCITE-306] Standardize code style for "import package.*; "

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 8e85e28..8cf2996 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
@@ -14,18 +14,24 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.metadata;
+package org.apache.calcite.rel.metadata;
 
-import java.lang.reflect.*;
-import java.util.*;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.util.*;
-
-import net.hydromatic.optiq.BuiltinMethod;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.util.BuiltInMethod;
+import org.apache.calcite.util.ReflectiveVisitor;
+import org.apache.calcite.util.Util;
 
 import com.google.common.base.Function;
-import com.google.common.collect.*;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Maps;
+
+import java.lang.reflect.InvocationHandler;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.lang.reflect.Proxy;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.Map;
 
 /**
  * Implementation of the {@link RelMetadataProvider} interface that dispatches
@@ -74,17 +80,18 @@ public class ReflectiveRelMetadataProvider
    * methods with a preceding argument.
    *
    * <p>For example, {@link BuiltInMetadata.Selectivity} has a method
-   * {@link BuiltInMetadata.Selectivity#getSelectivity(org.eigenbase.rex.RexNode)}.
+   * {@link BuiltInMetadata.Selectivity#getSelectivity(org.apache.calcite.rex.RexNode)}.
    * A class</p>
    *
    * <blockquote><pre><code>
    * class RelMdSelectivity {
-   *   public Double getSelectivity(UnionRel rel, RexNode predicate) { ... }
-   *   public Double getSelectivity(FilterRel rel, RexNode predicate) { ... }
+   *   public Double getSelectivity(Union rel, RexNode predicate) { }
+   *   public Double getSelectivity(LogicalFilter rel, RexNode predicate) { }
    * </code></pre></blockquote>
    *
    * <p>provides implementations of selectivity for relational expressions
-   * that extend {@link UnionRel} or {@link FilterRel}.</p>
+   * that extend {@link org.apache.calcite.rel.logical.LogicalUnion}
+   * or {@link org.apache.calcite.rel.logical.LogicalFilter}.</p>
    */
   public static RelMetadataProvider reflectiveSource(Method method,
       final Object target) {
@@ -115,16 +122,17 @@ public class ReflectiveRelMetadataProvider
                         public Object invoke(Object proxy, Method method,
                             Object[] args) throws Throwable {
                           // Suppose we are an implementation of Selectivity
-                          // that wraps "filter", a FilterRel, Then we implement
+                          // that wraps "filter", a LogicalFilter. Then we
+                          // implement
                           //   Selectivity.selectivity(rex)
                           // by calling method
                           //   new SelectivityImpl().selectivity(filter, rex)
                           if (method.equals(
-                              BuiltinMethod.METADATA_REL.method)) {
+                              BuiltInMethod.METADATA_REL.method)) {
                             return rel;
                           }
                           if (method.equals(
-                              BuiltinMethod.OBJECT_TO_STRING.method)) {
+                              BuiltInMethod.OBJECT_TO_STRING.method)) {
                             return metadataClass0.getSimpleName() + "(" + rel
                                 + ")";
                           }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/metadata/RelColumnMapping.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelColumnMapping.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelColumnMapping.java
index 44df143..4511f74 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/RelColumnMapping.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelColumnMapping.java
@@ -14,11 +14,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.metadata;
+package org.apache.calcite.rel.metadata;
 
 /**
- * Mapping from an input column of a {@link org.eigenbase.rel.RelNode} to one
- * of its output columns.
+ * Mapping from an input column of a {@link org.apache.calcite.rel.RelNode} to
+ * one of its output columns.
  */
 public class RelColumnMapping {
   public RelColumnMapping(

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/metadata/RelColumnOrigin.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelColumnOrigin.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelColumnOrigin.java
index 139d534..92cd7fe 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/RelColumnOrigin.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelColumnOrigin.java
@@ -14,11 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.metadata;
+package org.apache.calcite.rel.metadata;
 
-import java.util.*;
-
-import org.eigenbase.relopt.*;
+import org.apache.calcite.plan.RelOptTable;
 
 /**
  * RelColumnOrigin is a data structure describing one of the origins of an

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/metadata/RelMdColumnOrigins.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdColumnOrigins.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdColumnOrigins.java
index 72c11a6..d821744 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdColumnOrigins.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdColumnOrigins.java
@@ -14,24 +14,35 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.metadata;
+package org.apache.calcite.rel.metadata;
 
-import java.util.*;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Aggregate;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.SetOp;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.core.TableFunctionScan;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexVisitor;
+import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.util.BuiltInMethod;
 
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.rex.*;
-
-import net.hydromatic.optiq.BuiltinMethod;
+import java.util.HashSet;
+import java.util.Set;
 
 /**
- * RelMdColumnOrigins supplies a default implementation of {@link
- * RelMetadataQuery#getColumnOrigins} for the standard logical algebra.
+ * RelMdColumnOrigins supplies a default implementation of
+ * {@link RelMetadataQuery#getColumnOrigins} for the standard logical algebra.
  */
 public class RelMdColumnOrigins {
   public static final RelMetadataProvider SOURCE =
       ReflectiveRelMetadataProvider.reflectiveSource(
-          BuiltinMethod.COLUMN_ORIGIN.method, new RelMdColumnOrigins());
+          BuiltInMethod.COLUMN_ORIGIN.method, new RelMdColumnOrigins());
 
   //~ Constructors -----------------------------------------------------------
 
@@ -40,12 +51,12 @@ public class RelMdColumnOrigins {
   //~ Methods ----------------------------------------------------------------
 
   public Set<RelColumnOrigin> getColumnOrigins(
-      AggregateRelBase rel,
+      Aggregate rel,
       int iOutputColumn) {
     if (iOutputColumn < rel.getGroupCount()) {
       // Group columns pass through directly.
       return invokeGetColumnOrigins(
-          rel.getChild(),
+          rel.getInput(),
           iOutputColumn);
     }
 
@@ -57,7 +68,7 @@ public class RelMdColumnOrigins {
     for (Integer iInput : call.getArgList()) {
       Set<RelColumnOrigin> inputSet =
           invokeGetColumnOrigins(
-              rel.getChild(), iInput);
+              rel.getInput(), iInput);
       inputSet = createDerivedColumnOrigins(inputSet);
       if (inputSet != null) {
         set.addAll(inputSet);
@@ -67,7 +78,7 @@ public class RelMdColumnOrigins {
   }
 
   public Set<RelColumnOrigin> getColumnOrigins(
-      JoinRelBase rel,
+      Join rel,
       int iOutputColumn) {
     int nLeftColumns = rel.getLeft().getRowType().getFieldList().size();
     Set<RelColumnOrigin> set;
@@ -98,7 +109,7 @@ public class RelMdColumnOrigins {
   }
 
   public Set<RelColumnOrigin> getColumnOrigins(
-      SetOpRel rel,
+      SetOp rel,
       int iOutputColumn) {
     Set<RelColumnOrigin> set = new HashSet<RelColumnOrigin>();
     for (RelNode input : rel.getInputs()) {
@@ -115,9 +126,9 @@ public class RelMdColumnOrigins {
   }
 
   public Set<RelColumnOrigin> getColumnOrigins(
-      ProjectRelBase rel,
+      Project rel,
       int iOutputColumn) {
-    final RelNode child = rel.getChild();
+    final RelNode child = rel.getInput();
     RexNode rexNode = rel.getProjects().get(iOutputColumn);
 
     if (rexNode instanceof RexInputRef) {
@@ -150,23 +161,23 @@ public class RelMdColumnOrigins {
   }
 
   public Set<RelColumnOrigin> getColumnOrigins(
-      FilterRelBase rel,
+      Filter rel,
       int iOutputColumn) {
     return invokeGetColumnOrigins(
-        rel.getChild(),
+        rel.getInput(),
         iOutputColumn);
   }
 
   public Set<RelColumnOrigin> getColumnOrigins(
-      SortRel rel,
+      Sort rel,
       int iOutputColumn) {
     return invokeGetColumnOrigins(
-        rel.getChild(),
+        rel.getInput(),
         iOutputColumn);
   }
 
   public Set<RelColumnOrigin> getColumnOrigins(
-      TableFunctionRelBase rel,
+      TableFunctionScan rel,
       int iOutputColumn) {
     Set<RelColumnOrigin> set = new HashSet<RelColumnOrigin>();
     Set<RelColumnMapping> mappings = rel.getColumnMappings();

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/metadata/RelMdColumnUniqueness.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdColumnUniqueness.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdColumnUniqueness.java
index e47addf..262d9e5 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdColumnUniqueness.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdColumnUniqueness.java
@@ -14,27 +14,37 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.metadata;
+package org.apache.calcite.rel.metadata;
 
-import java.util.*;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Aggregate;
+import org.apache.calcite.rel.core.Correlator;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.JoinInfo;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.SemiJoin;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.BuiltInMethod;
 
-import org.eigenbase.rel.*;
-import org.eigenbase.rel.rules.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.rex.*;
-import org.eigenbase.sql.fun.*;
-
-import net.hydromatic.optiq.BuiltinMethod;
-import net.hydromatic.optiq.util.BitSets;
+import java.util.BitSet;
+import java.util.List;
 
 /**
- * RelMdColumnUniqueness supplies a default implementation of {@link
- * RelMetadataQuery#areColumnsUnique} for the standard logical algebra.
+ * RelMdColumnUniqueness supplies a default implementation of
+ * {@link RelMetadataQuery#areColumnsUnique} for the standard logical algebra.
  */
 public class RelMdColumnUniqueness {
   public static final RelMetadataProvider SOURCE =
       ReflectiveRelMetadataProvider.reflectiveSource(
-          BuiltinMethod.COLUMN_UNIQUENESS.method, new RelMdColumnUniqueness());
+          BuiltInMethod.COLUMN_UNIQUENESS.method, new RelMdColumnUniqueness());
 
   //~ Constructors -----------------------------------------------------------
 
@@ -43,27 +53,27 @@ public class RelMdColumnUniqueness {
   //~ Methods ----------------------------------------------------------------
 
   public Boolean areColumnsUnique(
-      FilterRelBase rel,
+      Filter rel,
       BitSet columns,
       boolean ignoreNulls) {
     return RelMetadataQuery.areColumnsUnique(
-        rel.getChild(),
+        rel.getInput(),
         columns,
         ignoreNulls);
   }
 
   public Boolean areColumnsUnique(
-      SortRel rel,
+      Sort rel,
       BitSet columns,
       boolean ignoreNulls) {
     return RelMetadataQuery.areColumnsUnique(
-        rel.getChild(),
+        rel.getInput(),
         columns,
         ignoreNulls);
   }
 
   public Boolean areColumnsUnique(
-      CorrelatorRel rel,
+      Correlator rel,
       BitSet columns,
       boolean ignoreNulls) {
     return RelMetadataQuery.areColumnsUnique(
@@ -73,10 +83,10 @@ public class RelMdColumnUniqueness {
   }
 
   public Boolean areColumnsUnique(
-      ProjectRelBase rel,
+      Project rel,
       BitSet columns,
       boolean ignoreNulls) {
-    // ProjectRel maps a set of rows to a different set;
+    // LogicalProject maps a set of rows to a different set;
     // Without knowledge of the mapping function(whether it
     // preserves uniqueness), it is only safe to derive uniqueness
     // info from the child of a project when the mapping is f(a) => a.
@@ -128,13 +138,13 @@ public class RelMdColumnUniqueness {
     }
 
     return RelMetadataQuery.areColumnsUnique(
-        rel.getChild(),
+        rel.getInput(),
         childColumns,
         ignoreNulls);
   }
 
   public Boolean areColumnsUnique(
-      JoinRelBase rel,
+      Join rel,
       BitSet columns, boolean
       ignoreNulls) {
     if (columns.cardinality() == 0) {
@@ -208,7 +218,7 @@ public class RelMdColumnUniqueness {
   }
 
   public Boolean areColumnsUnique(
-      SemiJoinRel rel,
+      SemiJoin rel,
       BitSet columns,
       boolean ignoreNulls) {
     // only return the unique keys from the LHS since a semijoin only
@@ -220,7 +230,7 @@ public class RelMdColumnUniqueness {
   }
 
   public Boolean areColumnsUnique(
-      AggregateRelBase rel,
+      Aggregate rel,
       BitSet columns,
       boolean ignoreNulls) {
     // group by keys form a unique key

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/metadata/RelMdDistinctRowCount.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdDistinctRowCount.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdDistinctRowCount.java
index b7637d8..e4d847d 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdDistinctRowCount.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdDistinctRowCount.java
@@ -14,28 +14,39 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.metadata;
+package org.apache.calcite.rel.metadata;
 
-import java.util.*;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Aggregate;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.SemiJoin;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.core.Union;
+import org.apache.calcite.rel.core.Values;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.BuiltInMethod;
+import org.apache.calcite.util.NumberUtil;
 
-import org.eigenbase.rel.*;
-import org.eigenbase.rel.rules.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.rex.*;
-import org.eigenbase.sql.fun.*;
-import org.eigenbase.util14.*;
-
-import net.hydromatic.optiq.BuiltinMethod;
-import net.hydromatic.optiq.util.BitSets;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.List;
 
 /**
- * RelMdDistinctRowCount supplies a default implementation of {@link
- * RelMetadataQuery#getDistinctRowCount} for the standard logical algebra.
+ * RelMdDistinctRowCount supplies a default implementation of
+ * {@link RelMetadataQuery#getDistinctRowCount} for the standard logical
+ * algebra.
  */
 public class RelMdDistinctRowCount {
   public static final RelMetadataProvider SOURCE =
       ReflectiveRelMetadataProvider.reflectiveSource(
-          BuiltinMethod.DISTINCT_ROW_COUNT.method, new RelMdDistinctRowCount());
+          BuiltInMethod.DISTINCT_ROW_COUNT.method, new RelMdDistinctRowCount());
 
   //~ Constructors -----------------------------------------------------------
 
@@ -44,7 +55,7 @@ public class RelMdDistinctRowCount {
   //~ Methods ----------------------------------------------------------------
 
   public Double getDistinctRowCount(
-      UnionRelBase rel,
+      Union rel,
       BitSet groupKey,
       RexNode predicate) {
     Double rowCount = 0.0;
@@ -78,17 +89,17 @@ public class RelMdDistinctRowCount {
   }
 
   public Double getDistinctRowCount(
-      SortRel rel,
+      Sort rel,
       BitSet groupKey,
       RexNode predicate) {
     return RelMetadataQuery.getDistinctRowCount(
-        rel.getChild(),
+        rel.getInput(),
         groupKey,
         predicate);
   }
 
   public Double getDistinctRowCount(
-      FilterRelBase rel,
+      Filter rel,
       BitSet groupKey,
       RexNode predicate) {
     // REVIEW zfong 4/18/06 - In the Broadbase code, duplicates are not
@@ -101,13 +112,13 @@ public class RelMdDistinctRowCount {
             rel.getCondition());
 
     return RelMetadataQuery.getDistinctRowCount(
-        rel.getChild(),
+        rel.getInput(),
         groupKey,
         unionPreds);
   }
 
   public Double getDistinctRowCount(
-      JoinRelBase rel,
+      Join rel,
       BitSet groupKey,
       RexNode predicate) {
     return RelMdUtil.getJoinDistinctRowCount(
@@ -119,7 +130,7 @@ public class RelMdDistinctRowCount {
   }
 
   public Double getDistinctRowCount(
-      SemiJoinRel rel,
+      SemiJoin rel,
       BitSet groupKey,
       RexNode predicate) {
     // create a RexNode representing the selectivity of the
@@ -141,7 +152,7 @@ public class RelMdDistinctRowCount {
   }
 
   public Double getDistinctRowCount(
-      AggregateRelBase rel,
+      Aggregate rel,
       BitSet groupKey,
       RexNode predicate) {
     // determine which predicates can be applied on the child of the
@@ -163,7 +174,7 @@ public class RelMdDistinctRowCount {
 
     Double distinctRowCount =
         RelMetadataQuery.getDistinctRowCount(
-            rel.getChild(),
+            rel.getInput(),
             childKey,
             childPreds);
     if (distinctRowCount == null) {
@@ -178,7 +189,7 @@ public class RelMdDistinctRowCount {
   }
 
   public Double getDistinctRowCount(
-      ValuesRelBase rel,
+      Values rel,
       BitSet groupKey,
       RexNode predicate) {
     Double selectivity = RelMdUtil.guessSelectivity(predicate);
@@ -189,7 +200,7 @@ public class RelMdDistinctRowCount {
   }
 
   public Double getDistinctRowCount(
-      ProjectRelBase rel,
+      Project rel,
       BitSet groupKey,
       RexNode predicate) {
     BitSet baseCols = new BitSet();
@@ -219,7 +230,7 @@ public class RelMdDistinctRowCount {
     }
     Double distinctRowCount =
         RelMetadataQuery.getDistinctRowCount(
-            rel.getChild(),
+            rel.getInput(),
             baseCols,
             modifiedPred);
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/metadata/RelMdExplainVisibility.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdExplainVisibility.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdExplainVisibility.java
index 68573b5..b0d926e 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdExplainVisibility.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdExplainVisibility.java
@@ -14,12 +14,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.metadata;
+package org.apache.calcite.rel.metadata;
 
-import org.eigenbase.rel.*;
-import org.eigenbase.sql.SqlExplainLevel;
-
-import net.hydromatic.optiq.BuiltinMethod;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.util.BuiltInMethod;
 
 /**
  * RelMdExplainVisibility supplies a default implementation of
@@ -28,7 +27,7 @@ import net.hydromatic.optiq.BuiltinMethod;
 public class RelMdExplainVisibility {
   public static final RelMetadataProvider SOURCE =
       ReflectiveRelMetadataProvider.reflectiveSource(
-          BuiltinMethod.EXPLAIN_VISIBILITY.method,
+          BuiltInMethod.EXPLAIN_VISIBILITY.method,
           new RelMdExplainVisibility());
 
   //~ Constructors -----------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/metadata/RelMdPercentageOriginalRows.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdPercentageOriginalRows.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdPercentageOriginalRows.java
index 7b4a677..991a488 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdPercentageOriginalRows.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdPercentageOriginalRows.java
@@ -14,18 +14,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.metadata;
+package org.apache.calcite.rel.metadata;
 
-import java.util.*;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
-
-import net.hydromatic.optiq.BuiltinMethod;
-import net.hydromatic.optiq.rules.java.JavaRules;
+import org.apache.calcite.adapter.enumerable.EnumerableInterpreter;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Aggregate;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.Union;
+import org.apache.calcite.util.BuiltInMethod;
 
 import com.google.common.collect.ImmutableList;
 
+import java.util.List;
+
 /**
  * RelMdPercentageOriginalRows supplies a default implementation of
  * {@link RelMetadataQuery#getPercentageOriginalRows} for the standard logical
@@ -39,28 +41,28 @@ public class RelMdPercentageOriginalRows {
       ChainedRelMetadataProvider.of(
           ImmutableList.of(
               ReflectiveRelMetadataProvider.reflectiveSource(
-                  BuiltinMethod.PERCENTAGE_ORIGINAL_ROWS.method, INSTANCE),
+                  BuiltInMethod.PERCENTAGE_ORIGINAL_ROWS.method, INSTANCE),
 
               ReflectiveRelMetadataProvider.reflectiveSource(
-                  BuiltinMethod.CUMULATIVE_COST.method, INSTANCE),
+                  BuiltInMethod.CUMULATIVE_COST.method, INSTANCE),
 
               ReflectiveRelMetadataProvider.reflectiveSource(
-                  BuiltinMethod.NON_CUMULATIVE_COST.method, INSTANCE)));
+                  BuiltInMethod.NON_CUMULATIVE_COST.method, INSTANCE)));
 
   //~ Methods ----------------------------------------------------------------
 
   private RelMdPercentageOriginalRows() {}
 
-  public Double getPercentageOriginalRows(AggregateRelBase rel) {
+  public Double getPercentageOriginalRows(Aggregate rel) {
     // REVIEW jvs 28-Mar-2006: The assumption here seems to be that
     // aggregation does not apply any filtering, so it does not modify the
     // percentage.  That's very much oversimplified.
 
     return RelMetadataQuery.getPercentageOriginalRows(
-        rel.getChild());
+        rel.getInput());
   }
 
-  public Double getPercentageOriginalRows(UnionRelBase rel) {
+  public Double getPercentageOriginalRows(Union rel) {
     double numerator = 0.0;
     double denominator = 0.0;
 
@@ -87,14 +89,14 @@ public class RelMdPercentageOriginalRows {
     return quotientForPercentage(numerator, denominator);
   }
 
-  public Double getPercentageOriginalRows(JoinRelBase rel) {
+  public Double getPercentageOriginalRows(Join rel) {
     // Assume any single-table filter conditions have already
     // been pushed down.
 
     // REVIEW jvs 28-Mar-2006: As with aggregation, this is
     // oversimplified.
 
-    // REVIEW jvs 28-Mar-2006:  need any special casing for SemiJoinRel?
+    // REVIEW jvs 28-Mar-2006:  need any special casing for SemiJoin?
 
     double left = RelMetadataQuery.getPercentageOriginalRows(rel.getLeft());
 
@@ -154,7 +156,7 @@ public class RelMdPercentageOriginalRows {
     return cost;
   }
 
-  public RelOptCost getCumulativeCost(JavaRules.EnumerableInterpreterRel rel) {
+  public RelOptCost getCumulativeCost(EnumerableInterpreter rel) {
     return RelMetadataQuery.getNonCumulativeCost(rel);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/metadata/RelMdPopulationSize.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdPopulationSize.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdPopulationSize.java
index 1755589..2afbc80 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdPopulationSize.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdPopulationSize.java
@@ -14,25 +14,32 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.metadata;
-
-import java.util.*;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.rel.rules.*;
-import org.eigenbase.rex.*;
-
-import net.hydromatic.optiq.BuiltinMethod;
-import net.hydromatic.optiq.util.BitSets;
+package org.apache.calcite.rel.metadata;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Aggregate;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.SemiJoin;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.core.Union;
+import org.apache.calcite.rel.core.Values;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.BuiltInMethod;
+
+import java.util.BitSet;
+import java.util.List;
 
 /**
- * RelMdPopulationSize supplies a default implementation of {@link
- * RelMetadataQuery#getPopulationSize} for the standard logical algebra.
+ * RelMdPopulationSize supplies a default implementation of
+ * {@link RelMetadataQuery#getPopulationSize} for the standard logical algebra.
  */
 public class RelMdPopulationSize {
   public static final RelMetadataProvider SOURCE =
       ReflectiveRelMetadataProvider.reflectiveSource(
-          BuiltinMethod.POPULATION_SIZE.method, new RelMdPopulationSize());
+          BuiltInMethod.POPULATION_SIZE.method, new RelMdPopulationSize());
 
   //~ Constructors -----------------------------------------------------------
 
@@ -40,19 +47,19 @@ public class RelMdPopulationSize {
 
   //~ Methods ----------------------------------------------------------------
 
-  public Double getPopulationSize(FilterRelBase rel, BitSet groupKey) {
+  public Double getPopulationSize(Filter rel, BitSet groupKey) {
     return RelMetadataQuery.getPopulationSize(
-        rel.getChild(),
+        rel.getInput(),
         groupKey);
   }
 
-  public Double getPopulationSize(SortRel rel, BitSet groupKey) {
+  public Double getPopulationSize(Sort rel, BitSet groupKey) {
     return RelMetadataQuery.getPopulationSize(
-        rel.getChild(),
+        rel.getInput(),
         groupKey);
   }
 
-  public Double getPopulationSize(UnionRelBase rel, BitSet groupKey) {
+  public Double getPopulationSize(Union rel, BitSet groupKey) {
     Double population = 0.0;
     for (RelNode input : rel.getInputs()) {
       Double subPop = RelMetadataQuery.getPopulationSize(input, groupKey);
@@ -64,30 +71,30 @@ public class RelMdPopulationSize {
     return population;
   }
 
-  public Double getPopulationSize(JoinRelBase rel, BitSet groupKey) {
+  public Double getPopulationSize(Join rel, BitSet groupKey) {
     return RelMdUtil.getJoinPopulationSize(rel, groupKey);
   }
 
-  public Double getPopulationSize(SemiJoinRel rel, BitSet groupKey) {
+  public Double getPopulationSize(SemiJoin rel, BitSet groupKey) {
     return RelMetadataQuery.getPopulationSize(
         rel.getLeft(),
         groupKey);
   }
 
-  public Double getPopulationSize(AggregateRelBase rel, BitSet groupKey) {
+  public Double getPopulationSize(Aggregate rel, BitSet groupKey) {
     BitSet childKey = new BitSet();
     RelMdUtil.setAggChildKeys(groupKey, rel, childKey);
     return RelMetadataQuery.getPopulationSize(
-        rel.getChild(),
+        rel.getInput(),
         childKey);
   }
 
-  public Double getPopulationSize(ValuesRelBase rel, BitSet groupKey) {
+  public Double getPopulationSize(Values rel, BitSet groupKey) {
     // assume half the rows are duplicates
     return rel.getRows() / 2;
   }
 
-  public Double getPopulationSize(ProjectRelBase rel, BitSet groupKey) {
+  public Double getPopulationSize(Project rel, BitSet groupKey) {
     BitSet baseCols = new BitSet();
     BitSet projCols = new BitSet();
     List<RexNode> projExprs = rel.getProjects();
@@ -95,7 +102,7 @@ public class RelMdPopulationSize {
 
     Double population =
         RelMetadataQuery.getPopulationSize(
-            rel.getChild(),
+            rel.getInput(),
             baseCols);
     if (population == null) {
       return null;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/metadata/RelMdPredicates.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdPredicates.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdPredicates.java
index eec7dd8..e15f074 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdPredicates.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdPredicates.java
@@ -14,7 +14,42 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.metadata;
+package org.apache.calcite.rel.metadata;
+
+import org.apache.calcite.linq4j.Linq4j;
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.linq4j.function.Predicate1;
+import org.apache.calcite.plan.RelOptPredicateList;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Aggregate;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.SemiJoin;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.rel.core.Union;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexPermuteInputsShuttle;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.BuiltInMethod;
+import org.apache.calcite.util.mapping.Mapping;
+import org.apache.calcite.util.mapping.MappingType;
+import org.apache.calcite.util.mapping.Mappings;
+
+import com.google.common.base.Function;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Iterators;
+import com.google.common.collect.Lists;
 
 import java.util.ArrayList;
 import java.util.BitSet;
@@ -27,48 +62,11 @@ import java.util.Set;
 import java.util.SortedMap;
 import java.util.TreeMap;
 
-import org.eigenbase.rel.AggregateRelBase;
-import org.eigenbase.rel.FilterRelBase;
-import org.eigenbase.rel.JoinRelBase;
-import org.eigenbase.rel.JoinRelType;
-import org.eigenbase.rel.ProjectRelBase;
-import org.eigenbase.rel.RelNode;
-import org.eigenbase.rel.SortRel;
-import org.eigenbase.rel.TableAccessRelBase;
-import org.eigenbase.rel.UnionRelBase;
-import org.eigenbase.rel.rules.SemiJoinRel;
-import org.eigenbase.relopt.RelOptPredicateList;
-import org.eigenbase.relopt.RelOptUtil;
-import org.eigenbase.rex.RexBuilder;
-import org.eigenbase.rex.RexCall;
-import org.eigenbase.rex.RexInputRef;
-import org.eigenbase.rex.RexNode;
-import org.eigenbase.rex.RexPermuteInputsShuttle;
-import org.eigenbase.rex.RexUtil;
-import org.eigenbase.rex.RexVisitorImpl;
-import org.eigenbase.sql.SqlKind;
-import org.eigenbase.util.mapping.Mapping;
-import org.eigenbase.util.mapping.MappingType;
-import org.eigenbase.util.mapping.Mappings;
-
-import net.hydromatic.linq4j.Linq4j;
-import net.hydromatic.linq4j.Ord;
-import net.hydromatic.linq4j.function.Predicate1;
-
-import net.hydromatic.optiq.BuiltinMethod;
-import net.hydromatic.optiq.util.BitSets;
-
-import com.google.common.base.Function;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Iterators;
-import com.google.common.collect.Lists;
-
 /**
  * Utility to infer Predicates that are applicable above a RelNode.
  *
  * <p>This is currently used by
- * {@link org.eigenbase.rel.rules.TransitivePredicatesOnJoinRule} to
+ * {@link org.apache.calcite.rel.rules.JoinPushTransitivePredicatesRule} to
  * infer <em>Predicates</em> that can be inferred from one side of a Join
  * to the other.
  *
@@ -111,7 +109,7 @@ import com.google.common.collect.Lists;
  */
 public class RelMdPredicates {
   public static final RelMetadataProvider SOURCE = ReflectiveRelMetadataProvider
-      .reflectiveSource(BuiltinMethod.PREDICATES.method, new RelMdPredicates());
+      .reflectiveSource(BuiltInMethod.PREDICATES.method, new RelMdPredicates());
 
   private static final List<RexNode> EMPTY_LIST = ImmutableList.of();
 
@@ -123,7 +121,7 @@ public class RelMdPredicates {
   /**
    * Infers predicates for a table scan.
    */
-  public RelOptPredicateList getPredicates(TableAccessRelBase table) {
+  public RelOptPredicateList getPredicates(TableScan table) {
     return RelOptPredicateList.EMPTY;
   }
 
@@ -146,8 +144,8 @@ public class RelMdPredicates {
    *
    * </ol>
    */
-  public RelOptPredicateList getPredicates(ProjectRelBase project) {
-    RelNode child = project.getChild();
+  public RelOptPredicateList getPredicates(Project project) {
+    RelNode child = project.getInput();
     RelOptPredicateList childInfo =
         RelMetadataQuery.getPulledUpPredicates(child);
 
@@ -181,8 +179,8 @@ public class RelMdPredicates {
   /**
    * Add the Filter condition to the pulledPredicates list from the child.
    */
-  public RelOptPredicateList getPredicates(FilterRelBase filter) {
-    RelNode child = filter.getChild();
+  public RelOptPredicateList getPredicates(Filter filter) {
+    RelNode child = filter.getInput();
     RelOptPredicateList childInfo =
         RelMetadataQuery.getPulledUpPredicates(child);
 
@@ -191,15 +189,15 @@ public class RelMdPredicates {
             RelOptUtil.conjunctions(filter.getCondition())));
   }
 
-  /** Infers predicates for a {@link SemiJoinRel}. */
-  public RelOptPredicateList getPredicates(SemiJoinRel semiJoin) {
+  /** Infers predicates for a {@link org.apache.calcite.rel.core.SemiJoin}. */
+  public RelOptPredicateList getPredicates(SemiJoin semiJoin) {
     // Workaround, pending [CALCITE-390] "Transitive inference (RelMdPredicate)
     // doesn't handle semi-join"
     return RelOptPredicateList.EMPTY;
   }
 
-  /** Infers predicates for a {@link JoinRelBase}. */
-  public RelOptPredicateList getPredicates(JoinRelBase join) {
+  /** Infers predicates for a {@link org.apache.calcite.rel.core.Join}. */
+  public RelOptPredicateList getPredicates(Join join) {
     RexBuilder rB = join.getCluster().getRexBuilder();
     RelNode left = join.getInput(0);
     RelNode right = join.getInput(1);
@@ -219,7 +217,7 @@ public class RelMdPredicates {
   }
 
   /**
-   * Infers predicates for an AggregateRel.
+   * Infers predicates for an Aggregate.
    *
    * <p>Pulls up predicates that only contains references to columns in the
    * GroupSet. For e.g.
@@ -230,8 +228,8 @@ public class RelMdPredicates {
    * pulledUpExprs    : { a &gt; 7}
    * </pre>
    */
-  public RelOptPredicateList getPredicates(AggregateRelBase agg) {
-    RelNode child = agg.getChild();
+  public RelOptPredicateList getPredicates(Aggregate agg) {
+    RelNode child = agg.getInput();
     RelOptPredicateList childInfo =
         RelMetadataQuery.getPulledUpPredicates(child);
 
@@ -257,11 +255,11 @@ public class RelMdPredicates {
   }
 
   /**
-   * Infers predicates for a UnionRelBase.
+   * Infers predicates for a Union.
    *
    * <p>The pulled up expression is a disjunction of its children's predicates.
    */
-  public RelOptPredicateList getPredicates(UnionRelBase union) {
+  public RelOptPredicateList getPredicates(Union union) {
     RexBuilder rB = union.getCluster().getRexBuilder();
     List<RexNode> orList = Lists.newArrayList();
     for (RelNode input : union.getInputs()) {
@@ -282,20 +280,28 @@ public class RelMdPredicates {
   }
 
   /**
-   * Infers predicates for a SortRel.
+   * Infers predicates for a Sort.
    */
-  public RelOptPredicateList getPredicates(SortRel sort) {
+  public RelOptPredicateList getPredicates(Sort sort) {
     RelNode child = sort.getInput(0);
     return RelMetadataQuery.getPulledUpPredicates(child);
   }
 
   /**
    * Utility to infer predicates from one side of the join that apply on the
-   * other side. Contract is: - initialize with a {@link JoinRelBase} and
-   * optional predicates applicable on its left and right subtrees. - you can
+   * other side.
+   *
+   * <p>Contract is:<ul>
+   *
+   * <li>initialize with a {@link org.apache.calcite.rel.core.Join} and
+   * optional predicates applicable on its left and right subtrees.
+   *
+   * <li>you can
    * then ask it for equivalentPredicate(s) given a predicate.
-   * <p>
-   * So for:
+   *
+   * </ul>
+   *
+   * <p>So for:
    * <ol>
    * <li>'<code>R1(x) join R2(y) on x = y</code>' a call for
    * equivalentPredicates on '<code>x > 7</code>' will return '
@@ -306,7 +312,7 @@ public class RelMdPredicates {
    * </ol>
    */
   static class JoinConditionBasedPredicateInference {
-    final JoinRelBase joinRel;
+    final Join joinRel;
     final int nSysFields;
     final int nFieldsLeft;
     final int nFieldsRight;
@@ -320,7 +326,7 @@ public class RelMdPredicates {
     final RexNode leftChildPredicates;
     final RexNode rightChildPredicates;
 
-    public JoinConditionBasedPredicateInference(JoinRelBase joinRel,
+    public JoinConditionBasedPredicateInference(Join joinRel,
         RexNode lPreds, RexNode rPreds) {
       super();
       this.joinRel = joinRel;
@@ -535,8 +541,7 @@ public class RelMdPredicates {
         super(true);
       }
 
-      @Override
-      public Void visitCall(RexCall call) {
+      @Override public Void visitCall(RexCall call) {
         if (call.getOperator().getKind() == SqlKind.EQUALS) {
           int lPos = pos(call.getOperands().get(0));
           int rPos = pos(call.getOperands().get(1));

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/metadata/RelMdRowCount.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdRowCount.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdRowCount.java
index 01df90e..2549570 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdRowCount.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdRowCount.java
@@ -14,29 +14,33 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.metadata;
+package org.apache.calcite.rel.metadata;
 
-import java.util.*;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Aggregate;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.SemiJoin;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.core.Union;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.util.BuiltInMethod;
+import org.apache.calcite.util.NumberUtil;
 
-import org.eigenbase.rel.*;
-import org.eigenbase.rel.rules.*;
-import org.eigenbase.rex.*;
-import org.eigenbase.util14.*;
-
-import net.hydromatic.optiq.BuiltinMethod;
+import java.util.BitSet;
 
 /**
- * RelMdRowCount supplies a default implementation of {@link
- * RelMetadataQuery#getRowCount} for the standard logical algebra.
+ * RelMdRowCount supplies a default implementation of
+ * {@link RelMetadataQuery#getRowCount} for the standard logical algebra.
  */
 public class RelMdRowCount {
   public static final RelMetadataProvider SOURCE =
       ReflectiveRelMetadataProvider.reflectiveSource(
-          BuiltinMethod.ROW_COUNT.method, new RelMdRowCount());
+          BuiltInMethod.ROW_COUNT.method, new RelMdRowCount());
 
   //~ Methods ----------------------------------------------------------------
 
-  public Double getRowCount(UnionRelBase rel) {
+  public Double getRowCount(Union rel) {
     double nRows = 0.0;
 
     for (RelNode input : rel.getInputs()) {
@@ -49,23 +53,23 @@ public class RelMdRowCount {
     return nRows;
   }
 
-  public Double getRowCount(FilterRelBase rel) {
+  public Double getRowCount(Filter rel) {
     return NumberUtil.multiply(
         RelMetadataQuery.getSelectivity(
-            rel.getChild(),
+            rel.getInput(),
             rel.getCondition()),
-        RelMetadataQuery.getRowCount(rel.getChild()));
+        RelMetadataQuery.getRowCount(rel.getInput()));
   }
 
-  public Double getRowCount(ProjectRelBase rel) {
-    return RelMetadataQuery.getRowCount(rel.getChild());
+  public Double getRowCount(Project rel) {
+    return RelMetadataQuery.getRowCount(rel.getInput());
   }
 
-  public Double getRowCount(SortRel rel) {
-    return RelMetadataQuery.getRowCount(rel.getChild());
+  public Double getRowCount(Sort rel) {
+    return RelMetadataQuery.getRowCount(rel.getInput());
   }
 
-  public Double getRowCount(SemiJoinRel rel) {
+  public Double getRowCount(SemiJoin rel) {
     // create a RexNode representing the selectivity of the
     // semijoin filter and pass it to getSelectivity
     RexNode semiJoinSelectivity =
@@ -78,7 +82,7 @@ public class RelMdRowCount {
         RelMetadataQuery.getRowCount(rel.getLeft()));
   }
 
-  public Double getRowCount(AggregateRelBase rel) {
+  public Double getRowCount(Aggregate rel) {
     BitSet groupKey = new BitSet();
     for (int i = 0; i < rel.getGroupCount(); i++) {
       groupKey.set(i);
@@ -87,11 +91,11 @@ public class RelMdRowCount {
     // rowcount is the cardinality of the group by columns
     Double distinctRowCount =
         RelMetadataQuery.getDistinctRowCount(
-            rel.getChild(),
+            rel.getInput(),
             groupKey,
             null);
     if (distinctRowCount == null) {
-      return RelMetadataQuery.getRowCount(rel.getChild()) / 10;
+      return RelMetadataQuery.getRowCount(rel.getInput()) / 10;
     } else {
       return distinctRowCount;
     }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/metadata/RelMdSelectivity.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdSelectivity.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdSelectivity.java
index 9cfbf1b..1706e13 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdSelectivity.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdSelectivity.java
@@ -14,27 +14,34 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.metadata;
-
-import java.util.*;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.rel.rules.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.rex.*;
-import org.eigenbase.sql.fun.*;
-
-import net.hydromatic.optiq.BuiltinMethod;
-import net.hydromatic.optiq.util.BitSets;
+package org.apache.calcite.rel.metadata;
+
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Aggregate;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.SemiJoin;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.core.Union;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.BuiltInMethod;
+
+import java.util.ArrayList;
+import java.util.List;
 
 /**
- * RelMdSelectivity supplies a default implementation of {@link
- * RelMetadataQuery#getSelectivity} for the standard logical algebra.
+ * RelMdSelectivity supplies a default implementation of
+ * {@link RelMetadataQuery#getSelectivity} for the standard logical algebra.
  */
 public class RelMdSelectivity {
   public static final RelMetadataProvider SOURCE =
       ReflectiveRelMetadataProvider.reflectiveSource(
-          BuiltinMethod.SELECTIVITY.method, new RelMdSelectivity());
+          BuiltInMethod.SELECTIVITY.method, new RelMdSelectivity());
 
   //~ Constructors -----------------------------------------------------------
 
@@ -43,7 +50,7 @@ public class RelMdSelectivity {
 
   //~ Methods ----------------------------------------------------------------
 
-  public Double getSelectivity(UnionRelBase rel, RexNode predicate) {
+  public Double getSelectivity(Union rel, RexNode predicate) {
     if ((rel.getInputs().size() == 0) || (predicate == null)) {
       return 1.0;
     }
@@ -78,32 +85,32 @@ public class RelMdSelectivity {
     return sumSelectedRows / sumRows;
   }
 
-  public Double getSelectivity(SortRel rel, RexNode predicate) {
+  public Double getSelectivity(Sort rel, RexNode predicate) {
     return RelMetadataQuery.getSelectivity(
-        rel.getChild(),
+        rel.getInput(),
         predicate);
   }
 
-  public Double getSelectivity(FilterRelBase rel, RexNode predicate) {
+  public Double getSelectivity(Filter rel, RexNode predicate) {
     // Take the difference between the predicate passed in and the
     // predicate in the filter's condition, so we don't apply the
     // selectivity of the filter twice.  If no predicate is passed in,
     // use the filter's condition.
     if (predicate != null) {
       return RelMetadataQuery.getSelectivity(
-          rel.getChild(),
+          rel.getInput(),
           RelMdUtil.minusPreds(
               rel.getCluster().getRexBuilder(),
               predicate,
               rel.getCondition()));
     } else {
       return RelMetadataQuery.getSelectivity(
-          rel.getChild(),
+          rel.getInput(),
           rel.getCondition());
     }
   }
 
-  public Double getSelectivity(SemiJoinRel rel, RexNode predicate) {
+  public Double getSelectivity(SemiJoin rel, RexNode predicate) {
     // create a RexNode representing the selectivity of the
     // semijoin filter and pass it to getSelectivity
     RexBuilder rexBuilder = rel.getCluster().getRexBuilder();
@@ -121,7 +128,7 @@ public class RelMdSelectivity {
         newPred);
   }
 
-  public Double getSelectivity(AggregateRelBase rel, RexNode predicate) {
+  public Double getSelectivity(Aggregate rel, RexNode predicate) {
     List<RexNode> notPushable = new ArrayList<RexNode>();
     List<RexNode> pushable = new ArrayList<RexNode>();
     RelOptUtil.splitFilters(
@@ -135,7 +142,7 @@ public class RelMdSelectivity {
 
     Double selectivity =
         RelMetadataQuery.getSelectivity(
-            rel.getChild(),
+            rel.getInput(),
             childPred);
     if (selectivity == null) {
       return null;
@@ -146,7 +153,7 @@ public class RelMdSelectivity {
     }
   }
 
-  public Double getSelectivity(ProjectRelBase rel, RexNode predicate) {
+  public Double getSelectivity(Project rel, RexNode predicate) {
     List<RexNode> notPushable = new ArrayList<RexNode>();
     List<RexNode> pushable = new ArrayList<RexNode>();
     RelOptUtil.splitFilters(
@@ -166,7 +173,7 @@ public class RelMdSelectivity {
     }
     Double selectivity =
         RelMetadataQuery.getSelectivity(
-            rel.getChild(),
+            rel.getInput(),
             modifiedPred);
     if (selectivity == null) {
       return null;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/metadata/RelMdUniqueKeys.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdUniqueKeys.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdUniqueKeys.java
index cdebe60..55083d8 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdUniqueKeys.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdUniqueKeys.java
@@ -14,25 +14,37 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.metadata;
-
-import java.util.*;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.rel.rules.*;
-import org.eigenbase.rex.*;
-
-import net.hydromatic.optiq.BuiltinMethod;
-import net.hydromatic.optiq.util.BitSets;
+package org.apache.calcite.rel.metadata;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Aggregate;
+import org.apache.calcite.rel.core.Correlator;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.JoinInfo;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.SemiJoin;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.BuiltInMethod;
+
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
 
 /**
- * RelMdUniqueKeys supplies a default implementation of {@link
- * RelMetadataQuery#getUniqueKeys} for the standard logical algebra.
+ * RelMdUniqueKeys supplies a default implementation of
+ * {@link RelMetadataQuery#getUniqueKeys} for the standard logical algebra.
  */
 public class RelMdUniqueKeys {
   public static final RelMetadataProvider SOURCE =
       ReflectiveRelMetadataProvider.reflectiveSource(
-          BuiltinMethod.UNIQUE_KEYS.method, new RelMdUniqueKeys());
+          BuiltInMethod.UNIQUE_KEYS.method, new RelMdUniqueKeys());
 
   //~ Constructors -----------------------------------------------------------
 
@@ -40,20 +52,20 @@ public class RelMdUniqueKeys {
 
   //~ Methods ----------------------------------------------------------------
 
-  public Set<BitSet> getUniqueKeys(FilterRelBase rel, boolean ignoreNulls) {
-    return RelMetadataQuery.getUniqueKeys(rel.getChild(), ignoreNulls);
+  public Set<BitSet> getUniqueKeys(Filter rel, boolean ignoreNulls) {
+    return RelMetadataQuery.getUniqueKeys(rel.getInput(), ignoreNulls);
   }
 
-  public Set<BitSet> getUniqueKeys(SortRel rel, boolean ignoreNulls) {
-    return RelMetadataQuery.getUniqueKeys(rel.getChild(), ignoreNulls);
+  public Set<BitSet> getUniqueKeys(Sort rel, boolean ignoreNulls) {
+    return RelMetadataQuery.getUniqueKeys(rel.getInput(), ignoreNulls);
   }
 
-  public Set<BitSet> getUniqueKeys(CorrelatorRel rel, boolean ignoreNulls) {
+  public Set<BitSet> getUniqueKeys(Correlator rel, boolean ignoreNulls) {
     return RelMetadataQuery.getUniqueKeys(rel.getLeft(), ignoreNulls);
   }
 
-  public Set<BitSet> getUniqueKeys(ProjectRelBase rel, boolean ignoreNulls) {
-    // ProjectRel maps a set of rows to a different set;
+  public Set<BitSet> getUniqueKeys(Project rel, boolean ignoreNulls) {
+    // LogicalProject maps a set of rows to a different set;
     // Without knowledge of the mapping function(whether it
     // preserves uniqueness), it is only safe to derive uniqueness
     // info from the child of a project when the mapping is f(a) => a.
@@ -81,7 +93,7 @@ public class RelMdUniqueKeys {
     }
 
     Set<BitSet> childUniqueKeySet =
-        RelMetadataQuery.getUniqueKeys(rel.getChild(), ignoreNulls);
+        RelMetadataQuery.getUniqueKeys(rel.getInput(), ignoreNulls);
 
     if (childUniqueKeySet != null) {
       // Now add to the projUniqueKeySet the child keys that are fully
@@ -108,7 +120,7 @@ public class RelMdUniqueKeys {
     return projUniqueKeySet;
   }
 
-  public Set<BitSet> getUniqueKeys(JoinRelBase rel, boolean ignoreNulls) {
+  public Set<BitSet> getUniqueKeys(Join rel, boolean ignoreNulls) {
     final RelNode left = rel.getLeft();
     final RelNode right = rel.getRight();
 
@@ -184,13 +196,13 @@ public class RelMdUniqueKeys {
     return retSet;
   }
 
-  public Set<BitSet> getUniqueKeys(SemiJoinRel rel, boolean ignoreNulls) {
+  public Set<BitSet> getUniqueKeys(SemiJoin rel, boolean ignoreNulls) {
     // only return the unique keys from the LHS since a semijoin only
     // returns the LHS
     return RelMetadataQuery.getUniqueKeys(rel.getLeft(), ignoreNulls);
   }
 
-  public Set<BitSet> getUniqueKeys(AggregateRelBase rel, boolean ignoreNulls) {
+  public Set<BitSet> getUniqueKeys(Aggregate rel, boolean ignoreNulls) {
     Set<BitSet> retSet = new HashSet<BitSet>();
 
     // group by keys form a unique key

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/metadata/RelMdUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdUtil.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdUtil.java
index 043c293..e3128fb 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdUtil.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdUtil.java
@@ -14,24 +14,41 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.metadata;
-
-import java.math.*;
-import java.util.*;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.rel.rules.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.rex.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.util.Bug;
-import org.eigenbase.util14.*;
-
-import net.hydromatic.optiq.util.BitSets;
+package org.apache.calcite.rel.metadata;
+
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Aggregate;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.core.JoinInfo;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.SemiJoin;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.sql.SqlFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.ReturnTypes;
+import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.Bug;
+import org.apache.calcite.util.NumberUtil;
 
 import com.google.common.collect.ImmutableList;
 
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
 /**
  * RelMdUtil provides utility methods used by the metadata provider methods.
  */
@@ -60,7 +77,7 @@ public class RelMdUtil {
    * @param rel the semijoin of interest
    * @return constructed rexnode
    */
-  public static RexNode makeSemiJoinSelectivityRexNode(SemiJoinRel rel) {
+  public static RexNode makeSemiJoinSelectivityRexNode(SemiJoin rel) {
     RexBuilder rexBuilder = rel.getCluster().getRexBuilder();
     double selectivity =
         computeSemiJoinSelectivity(
@@ -96,7 +113,7 @@ public class RelMdUtil {
    * @param rel semijoin rel
    * @return calculated selectivity
    */
-  public static double computeSemiJoinSelectivity(SemiJoinRel rel) {
+  public static double computeSemiJoinSelectivity(SemiJoin rel) {
     return computeSemiJoinSelectivity(
         rel.getLeft(),
         rel.getRight(),
@@ -118,7 +135,7 @@ public class RelMdUtil {
   public static double computeSemiJoinSelectivity(
       RelNode factRel,
       RelNode dimRel,
-      SemiJoinRel rel) {
+      SemiJoin rel) {
     return computeSemiJoinSelectivity(
         factRel,
         dimRel,
@@ -518,7 +535,7 @@ public class RelMdUtil {
    */
   public static void setAggChildKeys(
       BitSet groupKey,
-      AggregateRelBase aggRel,
+      Aggregate aggRel,
       BitSet childKey) {
     List<AggregateCall> aggCalls = aggRel.getAggCallList();
     for (int bit : BitSets.toIter(groupKey)) {
@@ -568,7 +585,7 @@ public class RelMdUtil {
    * @param expr projection expression
    * @return cardinality
    */
-  public static Double cardOfProjExpr(ProjectRelBase rel, RexNode expr) {
+  public static Double cardOfProjExpr(Project rel, RexNode expr) {
     return expr.accept(new CardOfProjExpr(rel));
   }
 
@@ -697,9 +714,9 @@ public class RelMdUtil {
   /** Visitor that walks over a scalar expression and computes the
    * cardinality of its result. */
   private static class CardOfProjExpr extends RexVisitorImpl<Double> {
-    private ProjectRelBase rel;
+    private Project rel;
 
-    public CardOfProjExpr(ProjectRelBase rel) {
+    public CardOfProjExpr(Project rel) {
       super(true);
       this.rel = rel;
     }
@@ -710,7 +727,7 @@ public class RelMdUtil {
       col.set(index);
       Double distinctRowCount =
           RelMetadataQuery.getDistinctRowCount(
-              rel.getChild(),
+              rel.getInput(),
               col,
               null);
       if (distinctRowCount == null) {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataProvider.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataProvider.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataProvider.java
index fd363da..0e1bf4f 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataProvider.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataProvider.java
@@ -14,9 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.metadata;
+package org.apache.calcite.rel.metadata;
 
-import org.eigenbase.rel.*;
+import org.apache.calcite.rel.RelNode;
 
 import com.google.common.base.Function;
 
@@ -43,10 +43,10 @@ public interface RelMetadataProvider {
    *
    * <blockquote><pre>
    * RelMetadataProvider provider;
-   * FilterRel filter;
+   * LogicalFilter filter;
    * RexNode predicate;
    * Function&lt;RelNode, Metadata&gt; function =
-   *   provider.apply(FilterRel.class, Selectivity.class};
+   *   provider.apply(LogicalFilter.class, Selectivity.class};
    * Selectivity selectivity = function.apply(filter);
    * Double d = selectivity.selectivity(predicate);
    * </pre></blockquote>

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQuery.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQuery.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQuery.java
index ed5ed48..4d86f79 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQuery.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQuery.java
@@ -14,30 +14,32 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.metadata;
+package org.apache.calcite.rel.metadata;
 
-import java.util.*;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.rex.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.stat.*;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPredicateList;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.SqlExplainLevel;
 
 import com.google.common.collect.Iterables;
 
+import java.util.BitSet;
+import java.util.Set;
+
 /**
- * RelMetadataQuery provides a strongly-typed facade on top of {@link
- * RelMetadataProvider} for the set of relational expression metadata queries
- * defined as standard within Eigenbase. The Javadoc on these methods serves as
- * their primary specification.
+ * RelMetadataQuery provides a strongly-typed facade on top of
+ * {@link RelMetadataProvider} for the set of relational expression metadata
+ * queries defined as standard within Calcite. The Javadoc on these methods
+ * serves as their primary specification.
  *
  * <p>To add a new standard query <code>Xyz</code> to this interface, follow
  * these steps:
  *
  * <ol>
  * <li>Add a static method <code>getXyz</code> specification to this class.
- * <li>Add unit tests to {@code org.eigenbase.test.RelMetadataTest}.
+ * <li>Add unit tests to {@code org.apache.calcite.test.RelMetadataTest}.
  * <li>Write a new provider class <code>RelMdXyz</code> in this package. Follow
  * the pattern from an existing class such as {@link RelMdColumnOrigins},
  * overloading on all of the logical relational expressions to which the query
@@ -65,22 +67,6 @@ public abstract class RelMetadataQuery {
   //~ Methods ----------------------------------------------------------------
 
   /**
-   * Returns statistics for a relational expression. These statistics include
-   * features such as row counts, or column distributions. Stats are typically
-   * collected by sampling a table. They might also be inferred from a rel's
-   * history. Certain rels, such as filters, might generate stats from their
-   * inputs.
-   *
-   * @param rel the relational expression.
-   * @return a statistics object, if statistics are available, or null
-   * otherwise
-   */
-  @Deprecated
-  public static RelStatSource getStatistics(RelNode rel) {
-    throw new UnsupportedOperationException();
-  }
-
-  /**
    * Returns the
    * {@link BuiltInMetadata.RowCount#getRowCount()}
    * statistic.
@@ -162,7 +148,7 @@ public abstract class RelMetadataQuery {
    * Determines the origin of a column, provided the column maps to a single
    * column that isn't derived.
    *
-   * @see #getColumnOrigins(org.eigenbase.rel.RelNode, int)
+   * @see #getColumnOrigins(org.apache.calcite.rel.RelNode, int)
    *
    * @param rel the RelNode of the column
    * @param column the offset of the column whose origin we are trying to
@@ -329,7 +315,7 @@ public abstract class RelMetadataQuery {
 
   /**
    * Returns the
-   * {@link org.eigenbase.rel.metadata.BuiltInMetadata.Predicates#getPredicates()}
+   * {@link org.apache.calcite.rel.metadata.BuiltInMetadata.Predicates#getPredicates()}
    * statistic.
    *
    * @param rel the relational expression

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/metadata/package-info.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/package-info.java b/core/src/main/java/org/apache/calcite/rel/metadata/package-info.java
index 6d865ff..ed98035 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/package-info.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/package-info.java
@@ -19,6 +19,6 @@
  * Defines metadata interfaces and utilities for relational
  * expressions.
  */
-package org.eigenbase.rel.metadata;
+package org.apache.calcite.rel.metadata;
 
 // End package-info.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/package-info.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/package-info.java b/core/src/main/java/org/apache/calcite/rel/package-info.java
index da6776e..fdbfd3b 100644
--- a/core/src/main/java/org/apache/calcite/rel/package-info.java
+++ b/core/src/main/java/org/apache/calcite/rel/package-info.java
@@ -20,20 +20,21 @@
  *
  * <h2>Related packages and classes</h2>
  * <ul>
- *     <li>Package<code> <a
- *             href="../sql/package-summary.html">org.eigenbase.sql</a></code>
- *         is an object model for SQL expressions
- *     </li>
- *     <li>Package<code> <a
- *             href="../sql/package-summary.html">org.eigenbase.rex</a></code>
- *         is an object model for relational row expressions
- *     </li>
- *     <li>Package<code> <a href="../relopt/package-summary.html">
- *         org.eigenbase.relopt</a></code>
- *         provides an optimizer interface.
- *     </li>
+ *
+ * <li>Package <code>
+ * <a href="../sql/package-summary.html">org.apache.calcite.sql</a></code>
+ * is an object model for SQL expressions
+ *
+ * <li>Package <code>
+ * <a href="../sql/package-summary.html">org.apache.calcite.rex</a></code>
+ * is an object model for relational row expressions
+ *
+ * <li>Package <code>
+ * <a href="../plan/package-summary.html">org.apache.calcite.plan</a></code>
+ * provides an optimizer interface.
+ *
  * </ul>
  */
-package org.eigenbase.rel;
+package org.apache.calcite.rel;
 
 // End package-info.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/rules/AggregateExpandDistinctAggregatesRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AggregateExpandDistinctAggregatesRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AggregateExpandDistinctAggregatesRule.java
index 39928cd..915ae87 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/AggregateExpandDistinctAggregatesRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/AggregateExpandDistinctAggregatesRule.java
@@ -14,39 +14,69 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
-
-import java.util.*;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.rex.*;
-import org.eigenbase.sql.fun.*;
-import org.eigenbase.util.*;
-
-import net.hydromatic.optiq.util.BitSets;
+package org.apache.calcite.rel.rules;
+
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Aggregate;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.core.RelFactories;
+import org.apache.calcite.rel.logical.LogicalAggregate;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Lists;
 
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
 /**
- * Rule to remove distinct aggregates from a {@link AggregateRel}.
+ * Planner rule that expands distinct aggregates
+ * (such as {@code COUNT(DISTINCT x)}) from a
+ * {@link org.apache.calcite.rel.logical.LogicalAggregate}.
+ *
+ * <p>How this is done depends upon the arguments to the function. If all
+ * functions have the same argument
+ * (e.g. {@code COUNT(DISTINCT x), SUM(DISTINCT x)} both have the argument
+ * {@code x}) then one extra {@link org.apache.calcite.rel.core.Aggregate} is
+ * sufficient.
+ *
+ * <p>If there are multiple arguments
+ * (e.g. {@code COUNT(DISTINCT x), COUNT(DISTINCT y)})
+ * the rule creates separate {@code Aggregate}s and combines using a
+ * {@link org.apache.calcite.rel.core.Join}.
  */
-public final class RemoveDistinctAggregateRule extends RelOptRule {
+public final class AggregateExpandDistinctAggregatesRule extends RelOptRule {
   //~ Static fields/initializers ---------------------------------------------
 
   /** The default instance of the rule; operates only on logical expressions. */
-  public static final RemoveDistinctAggregateRule INSTANCE =
-      new RemoveDistinctAggregateRule(AggregateRel.class,
+  public static final AggregateExpandDistinctAggregatesRule INSTANCE =
+      new AggregateExpandDistinctAggregatesRule(LogicalAggregate.class,
           RelFactories.DEFAULT_JOIN_FACTORY);
 
   private final RelFactories.JoinFactory joinFactory;
 
   //~ Constructors -----------------------------------------------------------
 
-  public RemoveDistinctAggregateRule(Class<? extends AggregateRel> clazz,
+  public AggregateExpandDistinctAggregatesRule(
+      Class<? extends LogicalAggregate> clazz,
       RelFactories.JoinFactory joinFactory) {
     super(operand(clazz, any()));
     this.joinFactory = joinFactory;
@@ -55,7 +85,7 @@ public final class RemoveDistinctAggregateRule extends RelOptRule {
   //~ Methods ----------------------------------------------------------------
 
   public void onMatch(RelOptRuleCall call) {
-    final AggregateRelBase aggregate = call.rel(0);
+    final Aggregate aggregate = call.rel(0);
     if (!aggregate.containsDistinctCall()) {
       return;
     }
@@ -125,9 +155,9 @@ public final class RemoveDistinctAggregateRule extends RelOptRule {
       rel = null;
     } else {
       rel =
-          new AggregateRel(
+          new LogicalAggregate(
               aggregate.getCluster(),
-              aggregate.getChild(),
+              aggregate.getInput(),
               groupSet,
               newAggCallList);
     }
@@ -149,7 +179,7 @@ public final class RemoveDistinctAggregateRule extends RelOptRule {
    * and no non-distinct aggregate functions.
    */
   private RelNode convertMonopole(
-      AggregateRelBase aggregate,
+      Aggregate aggregate,
       List<Integer> argList) {
     // For example,
     //    SELECT deptno, COUNT(DISTINCT sal), SUM(DISTINCT sal)
@@ -167,7 +197,7 @@ public final class RemoveDistinctAggregateRule extends RelOptRule {
     // Project the columns of the GROUP BY plus the arguments
     // to the agg function.
     Map<Integer, Integer> sourceOf = new HashMap<Integer, Integer>();
-    final AggregateRelBase distinct =
+    final Aggregate distinct =
         createSelectDistinct(aggregate, argList, sourceOf);
 
     // Create an aggregate on top, with the new aggregate list.
@@ -201,7 +231,7 @@ public final class RemoveDistinctAggregateRule extends RelOptRule {
    * @return Relational expression
    */
   private RelNode doRewrite(
-      AggregateRelBase aggregate,
+      Aggregate aggregate,
       RelNode left,
       List<Integer> argList,
       List<RexInputRef> refs) {
@@ -213,16 +243,16 @@ public final class RemoveDistinctAggregateRule extends RelOptRule {
       leftFields = left.getRowType().getFieldList();
     }
 
-    // AggregateRel(
+    // LogicalAggregate(
     //     child,
     //     {COUNT(DISTINCT 1), SUM(DISTINCT 1), SUM(2)})
     //
     // becomes
     //
-    // AggregateRel(
-    //     JoinRel(
+    // LogicalAggregate(
+    //     LogicalJoin(
     //         child,
-    //         AggregateRel(child, < all columns > {}),
+    //         LogicalAggregate(child, < all columns > {}),
     //         INNER,
     //         <f2 = f5>))
     //
@@ -257,7 +287,7 @@ public final class RemoveDistinctAggregateRule extends RelOptRule {
     // Project the columns of the GROUP BY plus the arguments
     // to the agg function.
     Map<Integer, Integer> sourceOf = new HashMap<Integer, Integer>();
-    final AggregateRelBase distinct =
+    final Aggregate distinct =
         createSelectDistinct(aggregate, argList, sourceOf);
 
     // Now compute the aggregate functions on top of the distinct dataset.
@@ -312,7 +342,7 @@ public final class RemoveDistinctAggregateRule extends RelOptRule {
       aggCallList.add(newAggCall);
     }
 
-    AggregateRelBase distinctAgg =
+    Aggregate distinctAgg =
         aggregate.copy(
             aggregate.getTraitSet(),
             distinct,
@@ -390,7 +420,8 @@ public final class RemoveDistinctAggregateRule extends RelOptRule {
   }
 
   /**
-   * Given an {@link AggregateRel} and the ordinals of the arguments to a
+   * Given an {@link org.apache.calcite.rel.logical.LogicalAggregate}
+   * and the ordinals of the arguments to a
    * particular call to an aggregate function, creates a 'select distinct'
    * relational expression which projects the group columns and those
    * arguments but nothing else.
@@ -424,13 +455,13 @@ public final class RemoveDistinctAggregateRule extends RelOptRule {
    * @return Aggregate relational expression which projects the required
    * columns
    */
-  private static AggregateRelBase createSelectDistinct(
-      AggregateRelBase aggregate,
+  private static Aggregate createSelectDistinct(
+      Aggregate aggregate,
       List<Integer> argList,
       Map<Integer, Integer> sourceOf) {
     final List<Pair<RexNode, String>> projects =
         new ArrayList<Pair<RexNode, String>>();
-    final RelNode child = aggregate.getChild();
+    final RelNode child = aggregate.getInput();
     final List<RelDataTypeField> childFields =
         child.getRowType().getFieldList();
     for (int i : BitSets.toIter(aggregate.getGroupSet())) {
@@ -457,4 +488,4 @@ public final class RemoveDistinctAggregateRule extends RelOptRule {
   }
 }
 
-// End RemoveDistinctAggregateRule.java
+// End AggregateExpandDistinctAggregatesRule.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/rules/AggregateFilterTransposeRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AggregateFilterTransposeRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AggregateFilterTransposeRule.java
index c7b4f73..4b909cc 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/AggregateFilterTransposeRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/AggregateFilterTransposeRule.java
@@ -14,34 +14,33 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
+package org.apache.calcite.rel.rules;
 
-import java.util.BitSet;
-import java.util.List;
-
-import org.eigenbase.rel.AggregateCall;
-import org.eigenbase.rel.AggregateRelBase;
-import org.eigenbase.rel.Aggregation;
-import org.eigenbase.rel.FilterRelBase;
-import org.eigenbase.rel.RelNode;
-import org.eigenbase.rel.metadata.RelMetadataQuery;
-import org.eigenbase.relopt.RelOptRule;
-import org.eigenbase.relopt.RelOptRuleCall;
-import org.eigenbase.relopt.RelOptUtil;
-import org.eigenbase.relopt.SubstitutionVisitor;
-import org.eigenbase.rex.RexNode;
-import org.eigenbase.rex.RexUtil;
-import org.eigenbase.util.mapping.Mappings;
-
-import net.hydromatic.optiq.util.BitSets;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.SubstitutionVisitor;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Aggregate;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.sql.SqlAggFunction;
+import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.mapping.Mappings;
 
 import com.google.common.base.Function;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 
+import java.util.BitSet;
+import java.util.List;
+
 /**
- * Planner rule that matches an {@link org.eigenbase.rel.AggregateRelBase}
- * on a {@link org.eigenbase.rel.FilterRelBase} and transposes them,
+ * Planner rule that matches an {@link org.apache.calcite.rel.core.Aggregate}
+ * on a {@link org.apache.calcite.rel.core.Filter} and transposes them,
  * pushing the aggregate below the filter.
  *
  * <p>In some cases, it is necessary to split the aggregate.
@@ -52,7 +51,7 @@ import com.google.common.collect.Lists;
  * evaluate. The main use of the rule is to match a query that has a filter
  * under an aggregate to an existing aggregate table.
  *
- * @see org.eigenbase.rel.rules.FilterAggregateTransposeRule
+ * @see org.apache.calcite.rel.rules.FilterAggregateTransposeRule
  */
 public class AggregateFilterTransposeRule extends RelOptRule {
   public static final AggregateFilterTransposeRule INSTANCE =
@@ -60,20 +59,20 @@ public class AggregateFilterTransposeRule extends RelOptRule {
 
   private AggregateFilterTransposeRule() {
     super(
-        operand(AggregateRelBase.class,
-            operand(FilterRelBase.class, any())));
+        operand(Aggregate.class,
+            operand(Filter.class, any())));
   }
 
   public void onMatch(RelOptRuleCall call) {
-    final AggregateRelBase aggregate = call.rel(0);
-    final FilterRelBase filter = call.rel(1);
+    final Aggregate aggregate = call.rel(0);
+    final Filter filter = call.rel(1);
 
     // Do the columns used by the filter appear in the output of the aggregate?
     final BitSet filterColumns =
         RelOptUtil.InputFinder.bits(filter.getCondition());
     final BitSet newGroupSet =
         BitSets.union(aggregate.getGroupSet(), filterColumns);
-    final RelNode input = filter.getChild();
+    final RelNode input = filter.getInput();
     final Boolean unique =
         RelMetadataQuery.areColumnsUnique(input, newGroupSet);
     if (unique != null && unique) {
@@ -82,7 +81,7 @@ public class AggregateFilterTransposeRule extends RelOptRule {
       // the rule fires forever: A-F => A-F-A => A-A-F-A => A-A-A-F-A => ...
       return;
     }
-    final AggregateRelBase newAggregate =
+    final Aggregate newAggregate =
         aggregate.copy(aggregate.getTraitSet(), input, newGroupSet,
             aggregate.getAggCallList());
     final Mappings.TargetMapping mapping = Mappings.target(
@@ -95,7 +94,7 @@ public class AggregateFilterTransposeRule extends RelOptRule {
         newGroupSet.cardinality());
     final RexNode newCondition =
         RexUtil.apply(mapping, filter.getCondition());
-    final FilterRelBase newFilter = filter.copy(filter.getTraitSet(),
+    final Filter newFilter = filter.copy(filter.getTraitSet(),
         newAggregate, newCondition);
     if (BitSets.contains(aggregate.getGroupSet(), filterColumns)) {
       // Everything needed by the filter is returned by the aggregate.
@@ -111,7 +110,7 @@ public class AggregateFilterTransposeRule extends RelOptRule {
       final List<AggregateCall> topAggCallList = Lists.newArrayList();
       int i = newGroupSet.cardinality();
       for (AggregateCall aggregateCall : aggregate.getAggCallList()) {
-        final Aggregation rollup =
+        final SqlAggFunction rollup =
             SubstitutionVisitor.getRollup(aggregateCall.getAggregation());
         if (rollup == null) {
           // This aggregate cannot be rolled up.
@@ -125,7 +124,7 @@ public class AggregateFilterTransposeRule extends RelOptRule {
             new AggregateCall(rollup, aggregateCall.isDistinct(),
                 ImmutableList.of(i++), aggregateCall.type, aggregateCall.name));
       }
-      final AggregateRelBase topAggregate =
+      final Aggregate topAggregate =
           aggregate.copy(aggregate.getTraitSet(), newFilter, topGroupSet,
               topAggCallList);
       call.transformTo(topAggregate);

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/rules/AggregateProjectMergeRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AggregateProjectMergeRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AggregateProjectMergeRule.java
index c97a8c9..8b1dafc 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/AggregateProjectMergeRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/AggregateProjectMergeRule.java
@@ -14,30 +14,29 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
+package org.apache.calcite.rel.rules;
 
-import java.util.BitSet;
-import java.util.List;
-
-import org.eigenbase.rel.AggregateCall;
-import org.eigenbase.rel.AggregateRelBase;
-import org.eigenbase.rel.ProjectRelBase;
-import org.eigenbase.rel.RelFactories;
-import org.eigenbase.rel.RelNode;
-import org.eigenbase.relopt.RelOptRule;
-import org.eigenbase.relopt.RelOptRuleCall;
-import org.eigenbase.relopt.RelOptUtil;
-import org.eigenbase.rex.RexInputRef;
-import org.eigenbase.rex.RexNode;
-
-import net.hydromatic.optiq.util.BitSets;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Aggregate;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.RelFactories;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.util.BitSets;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 
+import java.util.BitSet;
+import java.util.List;
+
 /**
- * Planner rule that recognizes a {@link org.eigenbase.rel.AggregateRelBase}
- * on top of a {@link org.eigenbase.rel.ProjectRelBase} and if possible
+ * Planner rule that recognizes a {@link org.apache.calcite.rel.core.Aggregate}
+ * on top of a {@link org.apache.calcite.rel.core.Project} and if possible
  * aggregate through the project or removes the project.
  *
  * <p>This is only possible when the grouping expressions and arguments to
@@ -53,21 +52,21 @@ public class AggregateProjectMergeRule extends RelOptRule {
   /** Private constructor. */
   private AggregateProjectMergeRule() {
     super(
-        operand(AggregateRelBase.class,
-            operand(ProjectRelBase.class, any())));
+        operand(Aggregate.class,
+            operand(Project.class, any())));
   }
 
   public void onMatch(RelOptRuleCall call) {
-    final AggregateRelBase aggregate = call.rel(0);
-    final ProjectRelBase project = call.rel(1);
+    final Aggregate aggregate = call.rel(0);
+    final Project project = call.rel(1);
     RelNode x = apply(aggregate, project);
     if (x != null) {
       call.transformTo(x);
     }
   }
 
-  public static RelNode apply(AggregateRelBase aggregate,
-      ProjectRelBase project) {
+  public static RelNode apply(Aggregate aggregate,
+      Project project) {
     final List<Integer> newKeys = Lists.newArrayList();
     for (int key : BitSets.toIter(aggregate.getGroupSet())) {
       final RexNode rex = project.getProjects().get(key);
@@ -96,8 +95,8 @@ public class AggregateProjectMergeRule extends RelOptRule {
     }
 
     final BitSet newGroupSet = BitSets.of(newKeys);
-    final AggregateRelBase newAggregate =
-        aggregate.copy(aggregate.getTraitSet(), project.getChild(), newGroupSet,
+    final Aggregate newAggregate =
+        aggregate.copy(aggregate.getTraitSet(), project.getInput(), newGroupSet,
             aggCalls.build());
 
     // Add a project if the group set is not in the same order or


[12/58] [abbrv] [partial] incubator-calcite git commit: [CALCITE-306] Standardize code style for "import package.*; "

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java b/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
index 2770950..95e5279 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
@@ -14,18 +14,23 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.test;
+package org.apache.calcite.test;
 
-import java.io.*;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.util.*;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelVisitor;
+import org.apache.calcite.rel.externalize.RelXmlWriter;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.util.Bug;
+import org.apache.calcite.util.TestUtil;
+import org.apache.calcite.util.Util;
 
 import org.junit.Test;
 
+import java.io.PrintWriter;
+import java.io.StringWriter;
+
 /**
- * Unit test for {@link org.eigenbase.sql2rel.SqlToRelConverter}.
+ * Unit test for {@link org.apache.calcite.sql2rel.SqlToRelConverter}.
  */
 public class SqlToRelConverterTest extends SqlToRelTestBase {
   //~ Methods ----------------------------------------------------------------
@@ -52,18 +57,18 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
   @Test public void testAliasList() {
     check(
         "select a + b from (\n"
-        + "  select deptno, 1 as one, name from dept\n"
-        + ") as d(a, b, c)\n"
-        + "where c like 'X%'",
+            + "  select deptno, 1 as one, name from dept\n"
+            + ") as d(a, b, c)\n"
+            + "where c like 'X%'",
         "${plan}");
   }
 
   @Test public void testAliasList2() {
     check(
         "select * from (\n"
-        + "  select a, b, c from (values (1, 2, 3)) as t (c, b, a)\n"
-        + ") join dept on dept.deptno = c\n"
-        + "order by c + a",
+            + "  select a, b, c from (values (1, 2, 3)) as t (c, b, a)\n"
+            + ") join dept on dept.deptno = c\n"
+            + "order by c + a",
         "${plan}");
   }
 
@@ -73,9 +78,9 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
   @Test public void testMultiAnd() {
     check(
         "select * from emp\n"
-        + "where deptno < 10\n"
-        + "and deptno > 5\n"
-        + "and (deptno = 8 or empno < 100)",
+            + "where deptno < 10\n"
+            + "and deptno > 5\n"
+            + "and (deptno = 8 or empno < 100)",
         "${plan}");
   }
 
@@ -92,7 +97,7 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
    */
   @Test public void testConditionOffByOne() {
     // Bug causes the plan to contain
-    //   JoinRel(condition=[=($9, $9)], joinType=[inner])
+    //   LogicalJoin(condition=[=($9, $9)], joinType=[inner])
     check(
         "SELECT * FROM emp JOIN dept on emp.deptno + 0 = dept.deptno",
         "${plan}");
@@ -113,7 +118,7 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
   @Test public void testJoinOnIn() {
     check(
         "select * from emp join dept\n"
-        + " on emp.deptno = dept.deptno and emp.empno in (1, 3)",
+            + " on emp.deptno = dept.deptno and emp.empno in (1, 3)",
         "${plan}");
   }
 
@@ -127,16 +132,16 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
   @Test public void testJoinUsingThreeWay() {
     check(
         "select *\n"
-        + "from emp as e\n"
-        + "join dept as d using (deptno)\n"
-        + "join emp as e2 using (empno)", "${plan}");
+            + "from emp as e\n"
+            + "join dept as d using (deptno)\n"
+            + "join emp as e2 using (empno)", "${plan}");
   }
 
   @Test public void testJoinUsingCompound() {
     check(
         "SELECT * FROM emp LEFT JOIN ("
-        + "SELECT *, deptno * 5 as empno FROM dept) "
-        + "USING (deptno,empno)",
+            + "SELECT *, deptno * 5 as empno FROM dept) "
+            + "USING (deptno,empno)",
         "${plan}");
   }
 
@@ -161,8 +166,8 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
   @Test public void testJoinWithUnion() {
     check(
         "select grade from "
-        + "(select empno from emp union select deptno from dept), "
-        + "salgrade",
+            + "(select empno from emp union select deptno from dept), "
+            + "salgrade",
         "${plan}");
   }
 
@@ -213,17 +218,17 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
     // Try to confuse it with spurious columns.
     check(
         "select name, foo from ("
-        + "select deptno, name, count(deptno) as foo "
-        + "from dept "
-        + "group by name, deptno, name)",
+            + "select deptno, name, count(deptno) as foo "
+            + "from dept "
+            + "group by name, deptno, name)",
         "${plan}");
   }
 
   @Test public void testAggDistinct() {
     check(
         "select deptno, sum(sal), sum(distinct sal), count(*) "
-        + "from emp "
-        + "group by deptno",
+            + "from emp "
+            + "group by deptno",
         "${plan}");
   }
 
@@ -360,9 +365,9 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
   @Test public void testOrderUnion() {
     check(
         "select empno, sal from emp "
-        + "union all "
-        + "select deptno, deptno from dept "
-        + "order by sal desc, empno asc",
+            + "union all "
+            + "select deptno, deptno from dept "
+            + "order by sal desc, empno asc",
         "${plan}");
   }
 
@@ -372,18 +377,18 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
     }
     check(
         "select empno, sal from emp "
-        + "union all "
-        + "select deptno, deptno from dept "
-        + "order by 2",
+            + "union all "
+            + "select deptno, deptno from dept "
+            + "order by 2",
         "${plan}");
   }
 
   @Test public void testOrderUnionExprs() {
     check(
         "select empno, sal from emp "
-        + "union all "
-        + "select deptno, deptno from dept "
-        + "order by empno * sal + 2",
+            + "union all "
+            + "select deptno, deptno from dept "
+            + "order by empno * sal + 2",
         "${plan}");
   }
 
@@ -417,31 +422,31 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
    * conditions</a>. */
   @Test public void testGroupAlias() {
     check("select \"$f2\", max(x), max(x + 1)\n"
-        + "from (values (1, 2)) as t(\"$f2\", x)\n"
-        + "group by \"$f2\"",
+            + "from (values (1, 2)) as t(\"$f2\", x)\n"
+            + "group by \"$f2\"",
         "${plan}");
   }
 
   @Test public void testOrderGroup() {
     check(
         "select deptno, count(*) "
-        + "from emp "
-        + "group by deptno "
-        + "order by deptno * sum(sal) desc, min(empno)",
+            + "from emp "
+            + "group by deptno "
+            + "order by deptno * sum(sal) desc, min(empno)",
         "${plan}");
   }
 
   @Test public void testCountNoGroup() {
     check(
         "select count(*), sum(sal)\n"
-        + "from emp\n"
-        + "where empno > 10",
+            + "from emp\n"
+            + "where empno > 10",
         "${plan}");
   }
 
   @Test public void testWith() {
     check("with emp2 as (select * from emp)\n"
-        + "select * from emp2",
+            + "select * from emp2",
         "${plan}");
   }
 
@@ -450,28 +455,28 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
    * WITH ... ORDER BY query gives AssertionError</a>. */
   @Test public void testWithOrder() {
     check("with emp2 as (select * from emp)\n"
-        + "select * from emp2 order by deptno",
+            + "select * from emp2 order by deptno",
         "${plan}");
   }
 
   @Test public void testWithUnionOrder() {
     check("with emp2 as (select empno, deptno as x from emp)\n"
-        + "select * from emp2\n"
-        + "union all\n"
-        + "select * from emp2\n"
-        + "order by empno + x",
+            + "select * from emp2\n"
+            + "union all\n"
+            + "select * from emp2\n"
+            + "order by empno + x",
         "${plan}");
   }
 
   @Test public void testWithUnion() {
     check("with emp2 as (select * from emp where deptno > 10)\n"
-      + "select empno from emp2 where deptno < 30 union all select deptno from emp",
+            + "select empno from emp2 where deptno < 30 union all select deptno from emp",
         "${plan}");
   }
 
   @Test public void testWithAlias() {
     check("with w(x, y) as (select * from dept where deptno > 10)\n"
-        + "select x from w where x < 30 union all select deptno from dept",
+            + "select x from w where x < 30 union all select deptno from dept",
         "${plan}");
   }
 
@@ -485,10 +490,10 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
 
   @Test public void testWithInsideWhereExistsDecorrelate() {
     tester.withDecorrelation(true).assertConvertsTo("select * from emp\n"
-        + "where exists (\n"
-        + "  with dept2 as (select * from dept where dept.deptno >= emp.deptno)\n"
-        + "  select 1 from dept2 where deptno <= emp.deptno)",
-      "${plan}");
+            + "where exists (\n"
+            + "  with dept2 as (select * from dept where dept.deptno >= emp.deptno)\n"
+            + "  select 1 from dept2 where deptno <= emp.deptno)",
+        "${plan}");
   }
 
   @Test public void testWithInsideScalarSubquery() {
@@ -520,10 +525,10 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
   @Test public void testSampleQuery() {
     check(
         "select * from (\n"
-        + " select * from emp as e tablesample substitute('DATASET1')\n"
-        + " join dept on e.deptno = dept.deptno\n"
-        + ") tablesample substitute('DATASET2')\n"
-        + "where empno > 5",
+            + " select * from emp as e tablesample substitute('DATASET1')\n"
+            + " join dept on e.deptno = dept.deptno\n"
+            + ") tablesample substitute('DATASET2')\n"
+            + "where empno > 5",
         "${plan}");
   }
 
@@ -536,10 +541,10 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
   @Test public void testSampleBernoulliQuery() {
     check(
         "select * from (\n"
-        + " select * from emp as e tablesample bernoulli(10) repeatable(1)\n"
-        + " join dept on e.deptno = dept.deptno\n"
-        + ") tablesample bernoulli(50) repeatable(99)\n"
-        + "where empno > 5",
+            + " select * from emp as e tablesample bernoulli(10) repeatable(1)\n"
+            + " join dept on e.deptno = dept.deptno\n"
+            + ") tablesample bernoulli(50) repeatable(99)\n"
+            + "where empno > 5",
         "${plan}");
   }
 
@@ -552,10 +557,10 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
   @Test public void testSampleSystemQuery() {
     check(
         "select * from (\n"
-        + " select * from emp as e tablesample system(10) repeatable(1)\n"
-        + " join dept on e.deptno = dept.deptno\n"
-        + ") tablesample system(50) repeatable(99)\n"
-        + "where empno > 5",
+            + " select * from emp as e tablesample system(10) repeatable(1)\n"
+            + " join dept on e.deptno = dept.deptno\n"
+            + ") tablesample system(50) repeatable(99)\n"
+            + "where empno > 5",
         "${plan}");
   }
 
@@ -600,9 +605,9 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
   @Test public void testCorrelationJoin() {
     check(
         "select *,"
-        + "         multiset(select * from emp where deptno=dept.deptno) "
-        + "               as empset"
-        + "      from dept",
+            + "         multiset(select * from emp where deptno=dept.deptno) "
+            + "               as empset"
+            + "      from dept",
         "${plan}");
   }
 
@@ -627,14 +632,14 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
   @Test public void testExistsCorrelatedLimit() {
     tester.withDecorrelation(false).assertConvertsTo(
         "select*from emp where exists (\n"
-        + "  select 1 from dept where emp.deptno=dept.deptno limit 1)",
+            + "  select 1 from dept where emp.deptno=dept.deptno limit 1)",
         "${plan}");
   }
 
   @Test public void testExistsCorrelatedLimitDecorrelate() {
     tester.withDecorrelation(true).assertConvertsTo(
         "select*from emp where exists (\n"
-        + "  select 1 from dept where emp.deptno=dept.deptno limit 1)",
+            + "  select 1 from dept where emp.deptno=dept.deptno limit 1)",
         "${plan}");
   }
 
@@ -646,23 +651,23 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
     // Go over the default threshold of 20 to force a subquery.
     check(
         "select empno from emp where deptno in"
-        + " (10, 20, 30, 40, 50, 60, 70, 80, 90, 100"
-        + ", 110, 120, 130, 140, 150, 160, 170, 180, 190"
-        + ", 200, 210, 220, 230)",
+            + " (10, 20, 30, 40, 50, 60, 70, 80, 90, 100"
+            + ", 110, 120, 130, 140, 150, 160, 170, 180, 190"
+            + ", 200, 210, 220, 230)",
         "${plan}");
   }
 
   @Test public void testInUncorrelatedSubquery() {
     check(
         "select empno from emp where deptno in"
-        + " (select deptno from dept)",
+            + " (select deptno from dept)",
         "${plan}");
   }
 
   @Test public void testNotInUncorrelatedSubquery() {
     check(
         "select empno from emp where deptno not in"
-        + " (select deptno from dept)",
+            + " (select deptno from dept)",
         "${plan}");
   }
 
@@ -672,8 +677,8 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
     // WHERE clause -- so the translation is more complicated.
     check(
         "select name, deptno in (\n"
-        + "  select case when true then deptno else null end from emp)\n"
-        + "from dept",
+            + "  select case when true then deptno else null end from emp)\n"
+            + "from dept",
         "${plan}");
   }
 
@@ -682,8 +687,8 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
   @Test public void testNotInUncorrelatedSubqueryInSelect() {
     check(
         "select empno, deptno not in (\n"
-        + "  select case when true then deptno else null end from dept)\n"
-        + "from emp",
+            + "  select case when true then deptno else null end from dept)\n"
+            + "from emp",
         "${plan}");
   }
 
@@ -692,8 +697,8 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
   @Test public void testNotInUncorrelatedSubqueryInSelectNotNull() {
     check(
         "select empno, deptno not in (\n"
-        + "  select deptno from dept)\n"
-        + "from emp",
+            + "  select deptno from dept)\n"
+            + "from emp",
         "${plan}");
   }
 
@@ -751,9 +756,9 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
     // union with values
     check(
         "values (10), (20)\n"
-        + "union all\n"
-        + "select 34 from emp\n"
-        + "union all values (30), (45 + 10)",
+            + "union all\n"
+            + "select 34 from emp\n"
+            + "union all values (30), (45 + 10)",
         "${plan}");
   }
 
@@ -761,9 +766,9 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
     // union of subquery, inside from list, also values
     check(
         "select deptno from emp as emp0 cross join\n"
-        + " (select empno from emp union all\n"
-        + "  select deptno from dept where deptno > 20 union all\n"
-        + "  values (45), (67))",
+            + " (select empno from emp union all\n"
+            + "  select deptno from dept where deptno > 20 union all\n"
+            + "  values (45), (67))",
         "${plan}");
   }
 
@@ -789,19 +794,20 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
   @Test public void testOverMultiple() {
     check(
         "select sum(sal) over w1,\n"
-        + "  sum(deptno) over w1,\n"
-        + "  sum(deptno) over w2\n"
-        + "from emp\n"
-        + "where deptno - sal > 999\n"
-        + "window w1 as (partition by job order by hiredate rows 2 preceding),\n"
-        + "  w2 as (partition by job order by hiredate rows 3 preceding disallow partial),\n"
-        + "  w3 as (partition by job order by hiredate range interval '1' second preceding)",
+            + "  sum(deptno) over w1,\n"
+            + "  sum(deptno) over w2\n"
+            + "from emp\n"
+            + "where deptno - sal > 999\n"
+            + "window w1 as (partition by job order by hiredate rows 2 preceding),\n"
+            + "  w2 as (partition by job order by hiredate rows 3 preceding disallow partial),\n"
+            + "  w3 as (partition by job order by hiredate range interval '1' second preceding)",
         "${plan}");
   }
 
   /**
-   * Test one of the custom conversions which is recognized by the class of
-   * the operator (in this case, {@link org.eigenbase.sql.fun.SqlCaseOperator}).
+   * Test one of the custom conversions which is recognized by the class of the
+   * operator (in this case,
+   * {@link org.apache.calcite.sql.fun.SqlCaseOperator}).
    */
   @Test public void testCase() {
     check(
@@ -811,8 +817,8 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
 
   /**
    * Tests one of the custom conversions which is recognized by the identity
-   * of the operator (in this case, {@link
-   * org.eigenbase.sql.fun.SqlStdOperatorTable#CHARACTER_LENGTH}).
+   * of the operator (in this case,
+   * {@link org.apache.calcite.sql.fun.SqlStdOperatorTable#CHARACTER_LENGTH}).
    */
   @Test public void testCharLength() {
     // Note that CHARACTER_LENGTH becomes CHAR_LENGTH.
@@ -827,9 +833,9 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
     // result back to match the type of x.
     check(
         "select sum(sal) over w1,\n"
-        + "  avg(sal) over w1\n"
-        + "from emp\n"
-        + "window w1 as (partition by job order by hiredate rows 2 preceding)",
+            + "  avg(sal) over w1\n"
+            + "from emp\n"
+            + "window w1 as (partition by job order by hiredate rows 2 preceding)",
         "${plan}");
   }
 
@@ -839,18 +845,18 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
     // isn't needed.
     check(
         "select sum(sal) over w1,\n"
-        + "  avg(CAST(sal as real)) over w1\n"
-        + "from emp\n"
-        + "window w1 as (partition by job order by hiredate rows 2 preceding)",
+            + "  avg(CAST(sal as real)) over w1\n"
+            + "from emp\n"
+            + "window w1 as (partition by job order by hiredate rows 2 preceding)",
         "${plan}");
   }
 
   @Test public void testOverCountStar() {
     check(
         "select count(sal) over w1,\n"
-        + "  count(*) over w1\n"
-        + "from emp\n"
-        + "window w1 as (partition by job order by hiredate rows 2 preceding)",
+            + "  count(*) over w1\n"
+            + "from emp\n"
+            + "window w1 as (partition by job order by hiredate rows 2 preceding)",
 
         "${plan}");
   }
@@ -861,14 +867,14 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
   @Test public void testOverOrderWindow() {
     check(
         "select last_value(deptno) over w\n"
-        + "from emp\n"
-        + "window w as (order by empno)",
+            + "from emp\n"
+            + "window w as (order by empno)",
         "${plan}");
 
     // Same query using inline window
     check(
         "select last_value(deptno) over (order by empno)\n"
-        + "from emp\n",
+            + "from emp\n",
         "${plan}");
   }
 
@@ -880,14 +886,14 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
     // Window contains only ORDER BY (implicitly CURRENT ROW).
     check(
         "select last_value(deptno) over w\n"
-        + "from emp\n"
-        + "window w as (order by empno rows 2 following)",
+            + "from emp\n"
+            + "window w as (order by empno rows 2 following)",
         "${plan}");
 
     // Same query using inline window
     check(
         "select last_value(deptno) over (order by empno rows 2 following)\n"
-        + "from emp\n",
+            + "from emp\n",
         "${plan}");
   }
 
@@ -910,25 +916,25 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
     rel.explain(planWriter);
     pw.flush();
     TestUtil.assertEqualsVerbose(
-        "<RelNode type=\"ProjectRel\">\n"
-        + "\t<Property name=\"EXPR$0\">\n"
-        + "\t\t+(1, 2)\t</Property>\n"
-        + "\t<Property name=\"EXPR$1\">\n"
-        + "\t\t3\t</Property>\n"
-        + "\t<Inputs>\n"
-        + "\t\t<RelNode type=\"ValuesRel\">\n"
-        + "\t\t\t<Property name=\"tuples\">\n"
-        + "\t\t\t\t[{ true }]\t\t\t</Property>\n"
-        + "\t\t\t<Inputs/>\n"
-        + "\t\t</RelNode>\n"
-        + "\t</Inputs>\n"
-        + "</RelNode>\n",
+        "<RelNode type=\"LogicalProject\">\n"
+            + "\t<Property name=\"EXPR$0\">\n"
+            + "\t\t+(1, 2)\t</Property>\n"
+            + "\t<Property name=\"EXPR$1\">\n"
+            + "\t\t3\t</Property>\n"
+            + "\t<Inputs>\n"
+            + "\t\t<RelNode type=\"LogicalValues\">\n"
+            + "\t\t\t<Property name=\"tuples\">\n"
+            + "\t\t\t\t[{ true }]\t\t\t</Property>\n"
+            + "\t\t\t<Inputs/>\n"
+            + "\t\t</RelNode>\n"
+            + "\t</Inputs>\n"
+            + "</RelNode>\n",
         Util.toLinux(sw.toString()));
   }
 
   /** Test case for
    * <a href="https://issues.apache.org/jira/browse/CALCITE-412">CALCITE-412</a>,
-   * "RelFieldTrimmer: when trimming SortRel, the collation and trait set don't
+   * "RelFieldTrimmer: when trimming Sort, the collation and trait set don't
    * match". */
   @Test public void testSortWithTrim() {
     tester.assertConvertsTo(

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/test/SqlToRelTestBase.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/SqlToRelTestBase.java b/core/src/test/java/org/apache/calcite/test/SqlToRelTestBase.java
index 52cec3b..ab2cd05 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlToRelTestBase.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlToRelTestBase.java
@@ -14,41 +14,61 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.test;
-
-import java.util.*;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.rex.RexBuilder;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.fun.*;
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.sql.validate.*;
-import org.eigenbase.sql2rel.*;
-import org.eigenbase.util.*;
-
-import net.hydromatic.linq4j.expressions.Expression;
-
-import net.hydromatic.optiq.prepare.Prepare;
+package org.apache.calcite.test;
+
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelOptSchema;
+import org.apache.calcite.plan.RelOptSchemaWithSampling;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.prepare.Prepare;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollationImpl;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.logical.LogicalTableScan;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperatorTable;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.parser.SqlParser;
+import org.apache.calcite.sql.type.SqlTypeFactoryImpl;
+import org.apache.calcite.sql.validate.SqlConformance;
+import org.apache.calcite.sql.validate.SqlMonotonicity;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql.validate.SqlValidatorCatalogReader;
+import org.apache.calcite.sql.validate.SqlValidatorImpl;
+import org.apache.calcite.sql.validate.SqlValidatorTable;
+import org.apache.calcite.sql2rel.RelFieldTrimmer;
+import org.apache.calcite.sql2rel.SqlToRelConverter;
+import org.apache.calcite.sql2rel.StandardConvertletTable;
+import org.apache.calcite.util.Util;
 
 import com.google.common.base.Function;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterables;
 
-import static org.junit.Assert.*;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 /**
  * SqlToRelTestBase is an abstract base for tests which involve conversion from
  * SQL to relational algebra.
  *
- * <p>SQL statements to be translated can use the schema defined in {@link
- * MockCatalogReader}; note that this is slightly different from Farrago's SALES
- * schema. If you get a parser or validator error from your test SQL, look down
- * in the stack until you see "Caused by", which will usually tell you the real
- * error.
+ * <p>SQL statements to be translated can use the schema defined in
+ * {@link MockCatalogReader}; note that this is slightly different from
+ * Farrago's SALES schema. If you get a parser or validator error from your test
+ * SQL, look down in the stack until you see "Caused by", which will usually
+ * tell you the real error.
  */
 public abstract class SqlToRelTestBase {
   //~ Static fields/initializers ---------------------------------------------
@@ -114,7 +134,7 @@ public abstract class SqlToRelTestBase {
 
     /**
      * Factory method for a
-     * {@link net.hydromatic.optiq.prepare.Prepare.CatalogReader}.
+     * {@link org.apache.calcite.prepare.Prepare.CatalogReader}.
      */
     Prepare.CatalogReader createCatalogReader(
         RelDataTypeFactory typeFactory);
@@ -279,6 +299,7 @@ public abstract class SqlToRelTestBase {
     public void registerRules(RelOptPlanner planner) throws Exception {
     }
 
+      /** Mock column set. */
     protected class MockColumnSet implements RelOptTable {
       private final List<String> names;
       private final RelDataType rowType;
@@ -325,7 +346,7 @@ public abstract class SqlToRelTestBase {
 
       public RelNode toRel(
           ToRelContext context) {
-        return new TableAccessRel(context.getCluster(), this);
+        return new LogicalTableScan(context.getCluster(), this);
       }
 
       public List<RelCollation> getCollationList() {
@@ -342,6 +363,7 @@ public abstract class SqlToRelTestBase {
     }
   }
 
+  /** Table that delegates to a given table. */
   private static class DelegatingRelOptTable implements RelOptTable {
     private final RelOptTable parent;
 
@@ -377,7 +399,7 @@ public abstract class SqlToRelTestBase {
     }
 
     public RelNode toRel(ToRelContext context) {
-      return new TableAccessRel(context.getCluster(), this);
+      return new LogicalTableScan(context.getCluster(), this);
     }
 
     public List<RelCollation> getCollationList() {
@@ -390,8 +412,8 @@ public abstract class SqlToRelTestBase {
   }
 
   /**
-   * Default implementation of {@link Tester}, using mock classes {@link
-   * MockRelOptSchema} and {@link MockRelOptPlanner}.
+   * Default implementation of {@link Tester}, using mock classes
+   * {@link MockRelOptSchema} and {@link MockRelOptPlanner}.
    */
   public static class TesterImpl implements Tester {
     private RelOptPlanner planner;
@@ -620,6 +642,7 @@ public abstract class SqlToRelTestBase {
     }
   }
 
+    /** Validator for testing. */
   private static class FarragoTestValidator extends SqlValidatorImpl {
     public FarragoTestValidator(
         SqlOperatorTable opTab,

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/test/java/org/apache/calcite/test/SqlValidatorFeatureTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/SqlValidatorFeatureTest.java b/core/src/test/java/org/apache/calcite/test/SqlValidatorFeatureTest.java
index bcc553e..46a2fb4 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlValidatorFeatureTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlValidatorFeatureTest.java
@@ -14,20 +14,29 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.test;
-
-import org.eigenbase.reltype.*;
-import org.eigenbase.resource.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.sql.test.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.sql.validate.*;
-import org.eigenbase.util.*;
+package org.apache.calcite.test;
+
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.runtime.CalciteContextException;
+import org.apache.calcite.runtime.CalciteException;
+import org.apache.calcite.runtime.Feature;
+import org.apache.calcite.sql.SqlOperatorTable;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.test.DefaultSqlTestFactory;
+import org.apache.calcite.sql.test.DelegatingSqlTestFactory;
+import org.apache.calcite.sql.test.SqlTestFactory;
+import org.apache.calcite.sql.test.SqlTester;
+import org.apache.calcite.sql.test.SqlTesterImpl;
+import org.apache.calcite.sql.type.SqlTypeFactoryImpl;
+import org.apache.calcite.sql.validate.SqlConformance;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql.validate.SqlValidatorCatalogReader;
+import org.apache.calcite.sql.validate.SqlValidatorImpl;
 
 import org.junit.Test;
 
-import static org.eigenbase.util.Static.RESOURCE;
+import static org.apache.calcite.util.Static.RESOURCE;
 
 /**
  * SqlValidatorFeatureTest verifies that features can be independently enabled
@@ -50,8 +59,7 @@ public class SqlValidatorFeatureTest extends SqlValidatorTestCase {
 
   //~ Methods ----------------------------------------------------------------
 
-  @Override
-  public SqlTester getTester() {
+  @Override public SqlTester getTester() {
     return new SqlTesterImpl(new FeatureTesterFactory());
   }
 
@@ -117,13 +125,13 @@ public class SqlValidatorFeatureTest extends SqlValidatorTestCase {
 
   //~ Inner Classes ----------------------------------------------------------
 
+  /** Factory for tester objects. */
   private class FeatureTesterFactory extends DelegatingSqlTestFactory {
     public FeatureTesterFactory() {
       super(DefaultSqlTestFactory.INSTANCE);
     }
 
-    @Override
-    public SqlValidator getValidator(SqlTestFactory factory) {
+    @Override public SqlValidator getValidator(SqlTestFactory factory) {
       final RelDataTypeFactory typeFactory =
           new SqlTypeFactoryImpl(RelDataTypeSystem.DEFAULT);
       SqlConformance conformance = (SqlConformance) get("conformance");
@@ -136,6 +144,7 @@ public class SqlValidatorFeatureTest extends SqlValidatorTestCase {
     }
   }
 
+  /** Extension to {@link SqlValidatorImpl} that validates features. */
   private class FeatureValidator extends SqlValidatorImpl {
     protected FeatureValidator(
         SqlOperatorTable opTab,
@@ -149,14 +158,14 @@ public class SqlValidatorFeatureTest extends SqlValidatorTestCase {
         Feature feature,
         SqlParserPos context) {
       if (feature.equals(disabledFeature)) {
-        EigenbaseException ex =
-            new EigenbaseException(
+        CalciteException ex =
+            new CalciteException(
                 FEATURE_DISABLED,
                 null);
         if (context == null) {
           throw ex;
         }
-        throw new EigenbaseContextException(
+        throw new CalciteContextException(
             "location",
             ex,
             context.getLineNum(),


[42/58] [abbrv] [partial] incubator-calcite git commit: [CALCITE-306] Standardize code style for "import package.*; "

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/jdbc/MetaImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/jdbc/MetaImpl.java b/core/src/main/java/org/apache/calcite/jdbc/MetaImpl.java
index ce807b7..8448062 100644
--- a/core/src/main/java/org/apache/calcite/jdbc/MetaImpl.java
+++ b/core/src/main/java/org/apache/calcite/jdbc/MetaImpl.java
@@ -14,40 +14,66 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.jdbc;
-
-import net.hydromatic.avatica.*;
-
-import net.hydromatic.linq4j.*;
-import net.hydromatic.linq4j.expressions.*;
-import net.hydromatic.linq4j.function.*;
-
-import net.hydromatic.optiq.*;
-import net.hydromatic.optiq.Table;
-import net.hydromatic.optiq.impl.AbstractTableQueryable;
-import net.hydromatic.optiq.impl.java.AbstractQueryableTable;
-import net.hydromatic.optiq.impl.java.JavaTypeFactory;
-import net.hydromatic.optiq.runtime.*;
-
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.SqlJdbcFunctionCall;
-import org.eigenbase.sql.parser.SqlParser;
-import org.eigenbase.util.Pair;
-import org.eigenbase.util.Util;
+package org.apache.calcite.jdbc;
+
+import org.apache.calcite.DataContext;
+import org.apache.calcite.adapter.java.AbstractQueryableTable;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.avatica.AvaticaParameter;
+import org.apache.calcite.avatica.AvaticaPrepareResult;
+import org.apache.calcite.avatica.AvaticaResultSet;
+import org.apache.calcite.avatica.AvaticaStatement;
+import org.apache.calcite.avatica.ColumnMetaData;
+import org.apache.calcite.avatica.Cursor;
+import org.apache.calcite.avatica.Meta;
+import org.apache.calcite.linq4j.Enumerable;
+import org.apache.calcite.linq4j.Enumerator;
+import org.apache.calcite.linq4j.Linq4j;
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.linq4j.QueryProvider;
+import org.apache.calcite.linq4j.Queryable;
+import org.apache.calcite.linq4j.function.Function1;
+import org.apache.calcite.linq4j.function.Functions;
+import org.apache.calcite.linq4j.function.Predicate1;
+import org.apache.calcite.linq4j.tree.Primitive;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeFactoryImpl;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.runtime.EnumeratorCursor;
+import org.apache.calcite.runtime.FlatLists;
+import org.apache.calcite.runtime.RecordEnumeratorCursor;
+import org.apache.calcite.schema.Schema;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.schema.Table;
+import org.apache.calcite.schema.impl.AbstractTableQueryable;
+import org.apache.calcite.sql.SqlJdbcFunctionCall;
+import org.apache.calcite.sql.parser.SqlParser;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.*;
+import com.google.common.base.CaseFormat;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Maps;
 
 import java.lang.reflect.Field;
 import java.lang.reflect.Modifier;
-import java.sql.*;
+import java.sql.DatabaseMetaData;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Time;
+import java.sql.Timestamp;
 import java.sql.Types;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
 import java.util.regex.Pattern;
 
 /**
  * Helper for implementing the {@code getXxx} methods such as
- * {@link net.hydromatic.avatica.AvaticaDatabaseMetaData#getTables}.
+ * {@link org.apache.calcite.avatica.AvaticaDatabaseMetaData#getTables}.
  */
 public class MetaImpl implements Meta {
   private static final Map<Class, Pair<Integer, String>> MAP =
@@ -74,9 +100,9 @@ public class MetaImpl implements Meta {
 
   static final Driver DRIVER = new Driver();
 
-  final OptiqConnectionImpl connection;
+  final CalciteConnectionImpl connection;
 
-  public MetaImpl(OptiqConnectionImpl connection) {
+  public MetaImpl(CalciteConnectionImpl connection) {
     this.connection = connection;
   }
 
@@ -158,7 +184,9 @@ public class MetaImpl implements Meta {
       if (Modifier.isPublic(field.getModifiers())
           && !Modifier.isStatic(field.getModifiers())) {
         list.add(
-            columnMetaData(Util.camelToUpper(field.getName()),
+            columnMetaData(
+                CaseFormat.LOWER_CAMEL.to(CaseFormat.UPPER_UNDERSCORE,
+                    field.getName()),
                 list.size() + 1, field.getType()));
       }
     }
@@ -169,37 +197,36 @@ public class MetaImpl implements Meta {
    * not implemented or which query entities that are not supported (e.g.
    * triggers in Lingual). */
   public static <E> ResultSet createEmptyResultSet(
-      OptiqConnectionImpl connection,
+      CalciteConnectionImpl connection,
       final Class<E> clazz) {
     return createResultSet(connection, ImmutableMap.<String, Object>of(),
         fieldMetaData(clazz),
         new RecordEnumeratorCursor<E>(Linq4j.<E>emptyEnumerator(), clazz));
   }
 
-  private static <E> ResultSet createResultSet(OptiqConnectionImpl connection,
+  private static <E> ResultSet createResultSet(CalciteConnectionImpl connection,
       final Map<String, Object> internalParameters,
       final ColumnMetaData.StructType structType,
       final Cursor cursor) {
     try {
       final AvaticaResultSet resultSet = connection.getFactory().newResultSet(
           connection.createStatement(),
-          new OptiqPrepare.PrepareResult<E>("",
+          new CalcitePrepare.PrepareResult<E>("",
               ImmutableList.<AvaticaParameter>of(), internalParameters, null,
               structType, -1, null, Object.class) {
-            @Override
-            public Cursor createCursor(DataContext dataContext) {
+            @Override public Cursor createCursor(DataContext dataContext) {
               return cursor;
             }
           },
           connection.getTimeZone());
-      return OptiqConnectionImpl.TROJAN.execute(resultSet);
+      return CalciteConnectionImpl.TROJAN.execute(resultSet);
     } catch (SQLException e) {
       throw new RuntimeException(e);
     }
   }
 
   private static ResultSet createResultSet(
-      OptiqConnectionImpl connection,
+      CalciteConnectionImpl connection,
       final Enumerable<?> enumerable,
       final NamedFieldGetter columnGetter) {
     //noinspection unchecked
@@ -334,12 +361,12 @@ public class MetaImpl implements Meta {
     return Linq4j.asEnumerable(
         connection.rootSchema.getSubSchemaMap().values())
         .select(
-            new Function1<OptiqSchema, MetaSchema>() {
-              public MetaSchema apply(OptiqSchema optiqSchema) {
+            new Function1<CalciteSchema, MetaSchema>() {
+              public MetaSchema apply(CalciteSchema calciteSchema) {
                 return new MetaSchema(
-                    optiqSchema,
+                    calciteSchema,
                     connection.getCatalog(),
-                    optiqSchema.getName());
+                    calciteSchema.getName());
               }
             })
         .orderBy(
@@ -363,12 +390,12 @@ public class MetaImpl implements Meta {
   }
 
   Enumerable<MetaTable> tables(final MetaSchema schema) {
-    return Linq4j.asEnumerable(schema.optiqSchema.getTableNames())
+    return Linq4j.asEnumerable(schema.calciteSchema.getTableNames())
         .select(
             new Function1<String, MetaTable>() {
               public MetaTable apply(String name) {
                 final Table table =
-                    schema.optiqSchema.getTable(name, true).getValue();
+                    schema.calciteSchema.getTable(name, true).getValue();
                 return new MetaTable(table,
                     schema.tableCatalog,
                     schema.tableSchem,
@@ -377,7 +404,7 @@ public class MetaImpl implements Meta {
             })
         .concat(
             Linq4j.asEnumerable(
-                schema.optiqSchema.getTablesBasedOnNullaryFunctions()
+                schema.calciteSchema.getTablesBasedOnNullaryFunctions()
                     .entrySet())
                 .select(
                     new Function1<Map.Entry<String, Table>, MetaTable>() {
@@ -405,7 +432,7 @@ public class MetaImpl implements Meta {
 
   public Enumerable<MetaColumn> columns(final MetaTable table) {
     final RelDataType rowType =
-        table.optiqTable.getRowType(connection.typeFactory);
+        table.calciteTable.getRowType(connection.typeFactory);
     return Linq4j.asEnumerable(rowType.getFieldList())
         .select(
             new Function1<RelDataTypeField, MetaColumn>() {
@@ -603,21 +630,22 @@ public class MetaImpl implements Meta {
   }
 
   public Cursor createCursor(AvaticaResultSet resultSet_) {
-    OptiqResultSet resultSet = (OptiqResultSet) resultSet_;
+    CalciteResultSet resultSet = (CalciteResultSet) resultSet_;
     Map<String, Object> map = Maps.newLinkedHashMap();
     final List<Object> parameterValues =
-        OptiqConnectionImpl.TROJAN.getParameterValues(resultSet.getStatement());
+        CalciteConnectionImpl.TROJAN.getParameterValues(
+            resultSet.getStatement());
     for (Ord<Object> o : Ord.zip(parameterValues)) {
       map.put("?" + o.i, o.e);
     }
     map.putAll(resultSet.getPrepareResult().getInternalParameters());
     final DataContext dataContext = connection.createDataContext(map);
-    OptiqPrepare.PrepareResult prepareResult = resultSet.getPrepareResult();
+    CalcitePrepare.PrepareResult prepareResult = resultSet.getPrepareResult();
     return prepareResult.createCursor(dataContext);
   }
 
   public AvaticaPrepareResult prepare(AvaticaStatement statement_, String sql) {
-    OptiqStatement statement = (OptiqStatement) statement_;
+    CalciteStatement statement = (CalciteStatement) statement_;
     int maxRowCount = statement.getMaxRows();
     return connection.parseQuery(sql,
         statement.createPrepareContext(),
@@ -626,14 +654,14 @@ public class MetaImpl implements Meta {
 
   /** A trojan-horse method, subject to change without notice. */
   @VisibleForTesting
-  public static DataContext createDataContext(OptiqConnection connection) {
-    return ((OptiqConnectionImpl) connection)
+  public static DataContext createDataContext(CalciteConnection connection) {
+    return ((CalciteConnectionImpl) connection)
         .createDataContext(ImmutableMap.<String, Object>of());
   }
 
   /** A trojan-horse method, subject to change without notice. */
   @VisibleForTesting
-  public static OptiqConnection connect(OptiqRootSchema schema,
+  public static CalciteConnection connect(CalciteRootSchema schema,
       JavaTypeFactory typeFactory) {
     return DRIVER.connect(schema, typeFactory);
   }
@@ -705,7 +733,7 @@ public class MetaImpl implements Meta {
 
   /** Metadata describing a table. */
   public static class MetaTable implements Named {
-    private final Table optiqTable;
+    private final Table calciteTable;
     public final String tableCat;
     public final String tableSchem;
     public final String tableName;
@@ -717,14 +745,14 @@ public class MetaImpl implements Meta {
     public final String selfReferencingColName = null;
     public final String refGeneration = null;
 
-    public MetaTable(Table optiqTable, String tableCat, String tableSchem,
+    public MetaTable(Table calciteTable, String tableCat, String tableSchem,
         String tableName) {
-      this.optiqTable = optiqTable;
-      assert optiqTable != null;
+      this.calciteTable = calciteTable;
+      assert calciteTable != null;
       this.tableCat = tableCat;
       this.tableSchem = tableSchem;
       this.tableName = tableName;
-      this.tableType = optiqTable.getJdbcTableType().name();
+      this.tableType = calciteTable.getJdbcTableType().name();
     }
 
     public String getName() {
@@ -734,15 +762,15 @@ public class MetaImpl implements Meta {
 
   /** Metadata describing a schema. */
   public static class MetaSchema implements Named {
-    private final OptiqSchema optiqSchema;
+    private final CalciteSchema calciteSchema;
     public final String tableCatalog;
     public final String tableSchem;
 
     public MetaSchema(
-        OptiqSchema optiqSchema,
+        CalciteSchema calciteSchema,
         String tableCatalog,
         String tableSchem) {
-      this.optiqSchema = optiqSchema;
+      this.calciteSchema = calciteSchema;
       this.tableCatalog = tableCatalog;
       this.tableSchem = tableSchem;
     }
@@ -907,7 +935,8 @@ public class MetaImpl implements Meta {
         List<ColumnMetaData> columns, List<Field> fields) {
       for (String name : names) {
         final int index = fields.size();
-        final String fieldName = Util.toCamelCase(name);
+        final String fieldName =
+            CaseFormat.UPPER_UNDERSCORE.to(CaseFormat.LOWER_CAMEL, name);
         final Field field;
         try {
           field = clazz.getField(fieldName);
@@ -962,8 +991,7 @@ public class MetaImpl implements Meta {
     }
 
     @SuppressWarnings("unchecked")
-    @Override
-    public Class<E> getElementType() {
+    @Override public Class<E> getElementType() {
       return (Class<E>) elementType;
     }
 
@@ -976,7 +1004,7 @@ public class MetaImpl implements Meta {
         @SuppressWarnings("unchecked")
         public Enumerator<T> enumerator() {
           return (Enumerator<T>) MetadataTable.this.enumerator(
-              ((OptiqConnectionImpl) queryProvider).meta());
+              ((CalciteConnectionImpl) queryProvider).meta());
         }
       };
     }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/jdbc/MetadataSchema.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/jdbc/MetadataSchema.java b/core/src/main/java/org/apache/calcite/jdbc/MetadataSchema.java
index b7d16dd..03e77aa 100644
--- a/core/src/main/java/org/apache/calcite/jdbc/MetadataSchema.java
+++ b/core/src/main/java/org/apache/calcite/jdbc/MetadataSchema.java
@@ -14,20 +14,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.jdbc;
+package org.apache.calcite.jdbc;
 
-import net.hydromatic.linq4j.Enumerable;
-import net.hydromatic.linq4j.Enumerator;
-import net.hydromatic.linq4j.function.Function1;
-
-import net.hydromatic.optiq.*;
-import net.hydromatic.optiq.impl.AbstractSchema;
+import org.apache.calcite.linq4j.Enumerable;
+import org.apache.calcite.linq4j.Enumerator;
+import org.apache.calcite.linq4j.function.Function1;
+import org.apache.calcite.schema.Schema;
+import org.apache.calcite.schema.Table;
+import org.apache.calcite.schema.impl.AbstractSchema;
 
 import com.google.common.collect.ImmutableMap;
 
 import java.util.Map;
 
-import static net.hydromatic.optiq.jdbc.MetaImpl.*;
+import static org.apache.calcite.jdbc.MetaImpl.MetaColumn;
+import static org.apache.calcite.jdbc.MetaImpl.MetaTable;
 
 /** Schema that contains metadata tables such as "TABLES" and "COLUMNS". */
 class MetadataSchema extends AbstractSchema {
@@ -57,8 +58,7 @@ class MetadataSchema extends AbstractSchema {
    * schema called "metadata" that contains tables "TABLES", "COLUMNS" etc. */
   private MetadataSchema() {}
 
-  @Override
-  protected Map<String, Table> getTableMap() {
+  @Override protected Map<String, Table> getTableMap() {
     return TABLE_MAP;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/jdbc/SqlTimeoutException.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/jdbc/SqlTimeoutException.java b/core/src/main/java/org/apache/calcite/jdbc/SqlTimeoutException.java
index 36f5174..c207842 100644
--- a/core/src/main/java/org/apache/calcite/jdbc/SqlTimeoutException.java
+++ b/core/src/main/java/org/apache/calcite/jdbc/SqlTimeoutException.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.jdbc;
+package org.apache.calcite.jdbc;
 
 import java.sql.SQLException;
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/jdbc/package-info.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/jdbc/package-info.java b/core/src/main/java/org/apache/calcite/jdbc/package-info.java
index 216ddca..40b3744 100644
--- a/core/src/main/java/org/apache/calcite/jdbc/package-info.java
+++ b/core/src/main/java/org/apache/calcite/jdbc/package-info.java
@@ -18,6 +18,6 @@
 /**
  * JDBC driver for Calcite.
  */
-package net.hydromatic.optiq.jdbc;
+package org.apache.calcite.jdbc;
 
 // End package-info.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/materialize/Lattice.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/materialize/Lattice.java b/core/src/main/java/org/apache/calcite/materialize/Lattice.java
index 24a2146..b4a75c3 100644
--- a/core/src/main/java/org/apache/calcite/materialize/Lattice.java
+++ b/core/src/main/java/org/apache/calcite/materialize/Lattice.java
@@ -14,40 +14,57 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.materialize;
-
-import net.hydromatic.optiq.Schemas;
-import net.hydromatic.optiq.Table;
-import net.hydromatic.optiq.impl.MaterializedViewTable;
-import net.hydromatic.optiq.impl.StarTable;
-import net.hydromatic.optiq.jdbc.OptiqPrepare;
-import net.hydromatic.optiq.jdbc.OptiqSchema;
-import net.hydromatic.optiq.prepare.OptiqPrepareImpl;
-import net.hydromatic.optiq.runtime.Utilities;
-import net.hydromatic.optiq.util.BitSets;
-import net.hydromatic.optiq.util.graph.*;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.RelOptUtil;
-import org.eigenbase.rex.*;
-import org.eigenbase.sql.SqlAggFunction;
-import org.eigenbase.sql.SqlDialect;
-import org.eigenbase.sql.SqlJoin;
-import org.eigenbase.sql.SqlKind;
-import org.eigenbase.sql.SqlNode;
-import org.eigenbase.sql.SqlSelect;
-import org.eigenbase.sql.SqlUtil;
-import org.eigenbase.sql.fun.SqlStdOperatorTable;
-import org.eigenbase.sql.validate.SqlValidatorUtil;
-import org.eigenbase.util.Util;
-import org.eigenbase.util.mapping.IntPair;
+package org.apache.calcite.materialize;
+
+import org.apache.calcite.jdbc.CalcitePrepare;
+import org.apache.calcite.jdbc.CalciteSchema;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.prepare.CalcitePrepareImpl;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.rel.logical.LogicalJoin;
+import org.apache.calcite.rel.logical.LogicalProject;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.runtime.Utilities;
+import org.apache.calcite.schema.Schemas;
+import org.apache.calcite.schema.Table;
+import org.apache.calcite.schema.impl.MaterializedViewTable;
+import org.apache.calcite.schema.impl.StarTable;
+import org.apache.calcite.sql.SqlAggFunction;
+import org.apache.calcite.sql.SqlDialect;
+import org.apache.calcite.sql.SqlJoin;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlSelect;
+import org.apache.calcite.sql.SqlUtil;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.validate.SqlValidatorUtil;
+import org.apache.calcite.util.BitSets;
+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 org.apache.calcite.util.mapping.IntPair;
 
 import com.google.common.base.Function;
 import com.google.common.base.Preconditions;
-import com.google.common.collect.*;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableListMultimap;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
 
 import java.math.BigInteger;
-import java.util.*;
+import java.util.BitSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
 
 /**
  * Structure that allows materialized views based upon a star schema to be
@@ -132,7 +149,7 @@ public class Lattice {
   }
 
   /** Creates a Lattice. */
-  public static Lattice create(OptiqSchema schema, String sql, boolean auto) {
+  public static Lattice create(CalciteSchema schema, String sql, boolean auto) {
     return builder(schema, sql).auto(auto).build();
   }
 
@@ -155,15 +172,15 @@ public class Lattice {
 
   private static boolean populate(List<RelNode> nodes, List<int[][]> tempLinks,
       RelNode rel) {
-    if (nodes.isEmpty() && rel instanceof ProjectRel) {
-      return populate(nodes, tempLinks, ((ProjectRel) rel).getChild());
+    if (nodes.isEmpty() && rel instanceof LogicalProject) {
+      return populate(nodes, tempLinks, ((LogicalProject) rel).getInput());
     }
-    if (rel instanceof TableAccessRelBase) {
+    if (rel instanceof TableScan) {
       nodes.add(rel);
       return true;
     }
-    if (rel instanceof JoinRel) {
-      JoinRel join = (JoinRel) rel;
+    if (rel instanceof LogicalJoin) {
+      LogicalJoin join = (LogicalJoin) rel;
       if (join.getJoinType() != JoinRelType.INNER) {
         throw new RuntimeException("only inner join allowed, but got "
             + join.getJoinType());
@@ -229,7 +246,7 @@ public class Lattice {
     if (usedNodes.isEmpty()) {
       usedNodes.add(nodes.get(0));
     }
-    final SqlDialect dialect = SqlDialect.DatabaseProduct.OPTIQ.getDialect();
+    final SqlDialect dialect = SqlDialect.DatabaseProduct.CALCITE.getDialect();
     final StringBuilder buf = new StringBuilder("SELECT ");
     final StringBuilder groupBuf = new StringBuilder("\nGROUP BY ");
     int k = 0;
@@ -300,7 +317,7 @@ public class Lattice {
         }
       }
     }
-    if (OptiqPrepareImpl.DEBUG) {
+    if (CalcitePrepareImpl.DEBUG) {
       System.out.println("Lattice SQL:\n" + buf);
     }
     buf.append(groupBuf);
@@ -324,8 +341,8 @@ public class Lattice {
     return StarTable.of(this, tables);
   }
 
-  public static Builder builder(OptiqSchema optiqSchema, String sql) {
-    return new Builder(optiqSchema, sql);
+  public static Builder builder(CalciteSchema calciteSchema, String sql) {
+    return new Builder(calciteSchema, sql);
   }
 
   public List<Measure> toMeasures(List<AggregateCall> aggCallList) {
@@ -426,14 +443,14 @@ public class Lattice {
    * (the fact table) have precisely one parent and an equi-join
    * condition on one or more pairs of columns linking to it. */
   public static class Node {
-    public final TableAccessRelBase scan;
+    public final TableScan scan;
     public final Node parent;
     public final ImmutableList<IntPair> link;
     public final int startCol;
     public final int endCol;
     public final String alias;
 
-    public Node(TableAccessRelBase scan, Node parent, List<IntPair> link,
+    public Node(TableScan scan, Node parent, List<IntPair> link,
         int startCol, int endCol, String alias) {
       this.scan = Preconditions.checkNotNull(scan);
       this.parent = parent;
@@ -473,10 +490,10 @@ public class Lattice {
 
   /** Measure in a lattice. */
   public static class Measure implements Comparable<Measure> {
-    public final Aggregation agg;
+    public final SqlAggFunction agg;
     public final ImmutableList<Column> args;
 
-    public Measure(Aggregation agg, Iterable<Column> args) {
+    public Measure(SqlAggFunction agg, Iterable<Column> args) {
       this.agg = Preconditions.checkNotNull(agg);
       this.args = ImmutableList.copyOf(args);
     }
@@ -494,7 +511,7 @@ public class Lattice {
     }
 
     @Override public int hashCode() {
-      return Util.hashV(agg, args);
+      return com.google.common.base.Objects.hashCode(agg, args);
     }
 
     @Override public boolean equals(Object obj) {
@@ -581,8 +598,8 @@ public class Lattice {
     private boolean auto = true;
     private Double rowCountEstimate;
 
-    public Builder(OptiqSchema schema, String sql) {
-      OptiqPrepare.ConvertResult parsed =
+    public Builder(CalciteSchema schema, String sql) {
+      CalcitePrepare.ConvertResult parsed =
           Schemas.convert(MaterializedViewTable.MATERIALIZATION_CONNECTION,
               schema, schema.path(null), sql);
 
@@ -626,7 +643,7 @@ public class Lattice {
             throw new RuntimeException("root node must not have relationships: "
                 + relNode);
           }
-          node = new Node((TableAccessRelBase) relNode, null, null,
+          node = new Node((TableScan) relNode, null, null,
               previousColumn, column, aliases.get(nodes.size()));
         } else {
           if (edges.size() != 1) {
@@ -634,7 +651,7 @@ public class Lattice {
                 "child node must have precisely one parent: " + relNode);
           }
           final Edge edge = edges.get(0);
-          node = new Node((TableAccessRelBase) relNode,
+          node = new Node((TableScan) relNode,
               map.get(edge.getSource()), edge.pairs, previousColumn, column,
               aliases.get(nodes.size()));
         }
@@ -693,7 +710,7 @@ public class Lattice {
     }
 
     /** Resolves the arguments of a
-     * {@link net.hydromatic.optiq.model.JsonMeasure}. They must either be null,
+     * {@link org.apache.calcite.model.JsonMeasure}. They must either be null,
      * a string, or a list of strings. Throws if the structure is invalid, or if
      * any of the columns do not exist in the lattice. */
     public ImmutableList<Column> resolveArgs(Object args) {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/materialize/MaterializationActor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/materialize/MaterializationActor.java b/core/src/main/java/org/apache/calcite/materialize/MaterializationActor.java
index 32193ef..ede840d 100644
--- a/core/src/main/java/org/apache/calcite/materialize/MaterializationActor.java
+++ b/core/src/main/java/org/apache/calcite/materialize/MaterializationActor.java
@@ -14,17 +14,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.materialize;
+package org.apache.calcite.materialize;
 
-import net.hydromatic.optiq.jdbc.OptiqRootSchema;
-import net.hydromatic.optiq.jdbc.OptiqSchema;
+import org.apache.calcite.jdbc.CalciteRootSchema;
+import org.apache.calcite.jdbc.CalciteSchema;
+import org.apache.calcite.rel.type.RelDataType;
 
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.util.Util;
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Multimap;
 
-import com.google.common.collect.*;
-
-import java.util.*;
+import java.util.List;
+import java.util.Map;
 
 /**
  * Actor that manages the state of materializations in the system.
@@ -49,8 +50,8 @@ class MaterializationActor {
    * same results as executing the query. */
   static class Materialization {
     final MaterializationKey key;
-    final OptiqRootSchema rootSchema;
-    OptiqSchema.TableEntry materializedTable;
+    final CalciteRootSchema rootSchema;
+    CalciteSchema.TableEntry materializedTable;
     final String sql;
     final RelDataType rowType;
 
@@ -66,8 +67,8 @@ class MaterializationActor {
      * @param rowType Row type
      */
     Materialization(MaterializationKey key,
-        OptiqRootSchema rootSchema,
-        OptiqSchema.TableEntry materializedTable,
+        CalciteRootSchema rootSchema,
+        CalciteSchema.TableEntry materializedTable,
         String sql,
         RelDataType rowType) {
       this.key = key;
@@ -82,10 +83,10 @@ class MaterializationActor {
    * schema, with the same path for resolving functions. */
   static class QueryKey {
     final String sql;
-    final OptiqSchema schema;
+    final CalciteSchema schema;
     final List<String> path;
 
-    QueryKey(String sql, OptiqSchema schema, List<String> path) {
+    QueryKey(String sql, CalciteSchema schema, List<String> path) {
       this.sql = sql;
       this.schema = schema;
       this.path = path;
@@ -100,7 +101,7 @@ class MaterializationActor {
     }
 
     @Override public int hashCode() {
-      return Util.hashV(sql, schema, path);
+      return com.google.common.base.Objects.hashCode(sql, schema, path);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/materialize/MaterializationKey.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/materialize/MaterializationKey.java b/core/src/main/java/org/apache/calcite/materialize/MaterializationKey.java
index af5faaa..c498f48 100644
--- a/core/src/main/java/org/apache/calcite/materialize/MaterializationKey.java
+++ b/core/src/main/java/org/apache/calcite/materialize/MaterializationKey.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.materialize;
+package org.apache.calcite.materialize;
 
 import java.io.Serializable;
 import java.util.UUID;
@@ -28,20 +28,17 @@ import java.util.UUID;
 public class MaterializationKey implements Serializable {
   private final UUID uuid = UUID.randomUUID();
 
-  @Override
-  public int hashCode() {
+  @Override public int hashCode() {
     return uuid.hashCode();
   }
 
-  @Override
-  public boolean equals(Object obj) {
+  @Override public boolean equals(Object obj) {
     return this == obj
         || obj instanceof MaterializationKey
         && uuid.equals(((MaterializationKey) obj).uuid);
   }
 
-  @Override
-  public String toString() {
+  @Override public String toString() {
     return uuid.toString();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/materialize/MaterializationService.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/materialize/MaterializationService.java b/core/src/main/java/org/apache/calcite/materialize/MaterializationService.java
index 596c7df..a4e289c 100644
--- a/core/src/main/java/org/apache/calcite/materialize/MaterializationService.java
+++ b/core/src/main/java/org/apache/calcite/materialize/MaterializationService.java
@@ -14,34 +14,47 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.materialize;
-
-import net.hydromatic.avatica.ColumnMetaData;
-
-import net.hydromatic.linq4j.*;
-import net.hydromatic.linq4j.expressions.Expression;
-import net.hydromatic.linq4j.function.Function1;
-import net.hydromatic.linq4j.function.Functions;
-
-import net.hydromatic.optiq.*;
-import net.hydromatic.optiq.Table;
-import net.hydromatic.optiq.config.OptiqConnectionProperty;
-import net.hydromatic.optiq.impl.clone.CloneSchema;
-import net.hydromatic.optiq.impl.java.JavaTypeFactory;
-import net.hydromatic.optiq.jdbc.*;
-import net.hydromatic.optiq.prepare.Prepare;
-import net.hydromatic.optiq.runtime.Hook;
-import net.hydromatic.optiq.util.BitSets;
-
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.reltype.RelDataTypeImpl;
-import org.eigenbase.util.Pair;
-import org.eigenbase.util.Util;
-
-import com.google.common.collect.*;
+package org.apache.calcite.materialize;
+
+import org.apache.calcite.DataContext;
+import org.apache.calcite.adapter.clone.CloneSchema;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.avatica.ColumnMetaData;
+import org.apache.calcite.config.CalciteConnectionProperty;
+import org.apache.calcite.jdbc.CalciteConnection;
+import org.apache.calcite.jdbc.CalcitePrepare;
+import org.apache.calcite.jdbc.CalciteSchema;
+import org.apache.calcite.jdbc.MetaImpl;
+import org.apache.calcite.linq4j.AbstractQueryable;
+import org.apache.calcite.linq4j.Enumerator;
+import org.apache.calcite.linq4j.QueryProvider;
+import org.apache.calcite.linq4j.function.Function1;
+import org.apache.calcite.linq4j.function.Functions;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.prepare.Prepare;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeImpl;
+import org.apache.calcite.runtime.Hook;
+import org.apache.calcite.schema.Schemas;
+import org.apache.calcite.schema.Table;
+import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
 
 import java.lang.reflect.Type;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.PriorityQueue;
 
 /**
  * Manages the collection of materialized tables known to the system,
@@ -54,16 +67,15 @@ public class MaterializationService {
   /** For testing. */
   private static final ThreadLocal<MaterializationService> THREAD_INSTANCE =
       new ThreadLocal<MaterializationService>() {
-        @Override
-        protected MaterializationService initialValue() {
+        @Override protected MaterializationService initialValue() {
           return new MaterializationService();
         }
       };
 
-  private static final Comparator<Pair<OptiqSchema.TableEntry, TileKey>> C =
-      new Comparator<Pair<OptiqSchema.TableEntry, TileKey>>() {
-        public int compare(Pair<OptiqSchema.TableEntry, TileKey> o0,
-            Pair<OptiqSchema.TableEntry, TileKey> o1) {
+  private static final Comparator<Pair<CalciteSchema.TableEntry, TileKey>> C =
+      new Comparator<Pair<CalciteSchema.TableEntry, TileKey>>() {
+        public int compare(Pair<CalciteSchema.TableEntry, TileKey> o0,
+            Pair<CalciteSchema.TableEntry, TileKey> o1) {
           // We prefer rolling up from the table with the fewest rows.
           final Table t0 = o0.left.getTable();
           final Table t1 = o1.left.getTable();
@@ -83,7 +95,7 @@ public class MaterializationService {
   }
 
   /** Defines a new materialization. Returns its key. */
-  public MaterializationKey defineMaterialization(final OptiqSchema schema,
+  public MaterializationKey defineMaterialization(final CalciteSchema schema,
       TileKey tileKey, String viewSql, List<String> viewSchemaPath,
       final String suggestedTableName, boolean create) {
     final MaterializationActor.QueryKey queryKey =
@@ -96,16 +108,16 @@ public class MaterializationService {
       return null;
     }
 
-    final OptiqConnection connection =
+    final CalciteConnection connection =
         MetaImpl.connect(schema.root(), null);
     final Pair<String, Table> pair = schema.getTableBySql(viewSql);
     Table materializedTable = pair == null ? null : pair.right;
     RelDataType rowType = null;
     if (materializedTable == null) {
-      final ImmutableMap<OptiqConnectionProperty, String> map =
-          ImmutableMap.of(OptiqConnectionProperty.CREATE_MATERIALIZATIONS,
+      final ImmutableMap<CalciteConnectionProperty, String> map =
+          ImmutableMap.of(CalciteConnectionProperty.CREATE_MATERIALIZATIONS,
               "false");
-      final OptiqPrepare.PrepareResult<Object> prepareResult =
+      final CalcitePrepare.PrepareResult<Object> prepareResult =
           Schemas.prepare(connection, schema, viewSchemaPath, viewSql, map);
       rowType = prepareResult.rowType;
       final JavaTypeFactory typeFactory = connection.getTypeFactory();
@@ -146,12 +158,12 @@ public class MaterializationService {
     }
     final String tableName =
         Schemas.uniqueTableName(schema, Util.first(suggestedTableName, "m"));
-    final OptiqSchema.TableEntry tableEntry =
+    final CalciteSchema.TableEntry tableEntry =
         schema.add(tableName, materializedTable, ImmutableList.of(viewSql));
     Hook.CREATE_MATERIALIZATION.run(tableName);
     if (rowType == null) {
       // If we didn't validate the SQL by populating a table, validate it now.
-      final OptiqPrepare.ParseResult parse =
+      final CalcitePrepare.ParseResult parse =
           Schemas.parse(connection, schema, viewSchemaPath, viewSql);
       rowType = parse.rowType;
     }
@@ -169,7 +181,7 @@ public class MaterializationService {
 
   /** Checks whether a materialization is valid, and if so, returns the table
    * where the data are stored. */
-  public OptiqSchema.TableEntry checkValid(MaterializationKey key) {
+  public CalciteSchema.TableEntry checkValid(MaterializationKey key) {
     final MaterializationActor.Materialization materialization =
         actor.keyMap.get(key);
     if (materialization != null) {
@@ -186,8 +198,8 @@ public class MaterializationService {
    * during the recursive SQL that populates a materialization. Otherwise a
    * materialization would try to create itself to populate itself!
    */
-  public Pair<OptiqSchema.TableEntry, TileKey> defineTile(Lattice lattice,
-      BitSet groupSet, List<Lattice.Measure> measureList, OptiqSchema schema,
+  public Pair<CalciteSchema.TableEntry, TileKey> defineTile(Lattice lattice,
+      BitSet groupSet, List<Lattice.Measure> measureList, CalciteSchema schema,
       boolean create, boolean exact) {
     MaterializationKey materializationKey;
     final TileKey tileKey =
@@ -196,7 +208,8 @@ public class MaterializationService {
     // Step 1. Look for an exact match for the tile.
     materializationKey = actor.keyByTile.get(tileKey);
     if (materializationKey != null) {
-      final OptiqSchema.TableEntry tableEntry = checkValid(materializationKey);
+      final CalciteSchema.TableEntry tableEntry =
+          checkValid(materializationKey);
       if (tableEntry != null) {
         return Pair.of(tableEntry, tileKey);
       }
@@ -211,7 +224,7 @@ public class MaterializationService {
       if (allSatisfiable(measureList, tileKey1)) {
         materializationKey = actor.keyByTile.get(tileKey1);
         if (materializationKey != null) {
-          final OptiqSchema.TableEntry tableEntry =
+          final CalciteSchema.TableEntry tableEntry =
               checkValid(materializationKey);
           if (tableEntry != null) {
             return Pair.of(tableEntry, tileKey1);
@@ -232,8 +245,8 @@ public class MaterializationService {
     // TODO: Use a partially-ordered set data structure, so we are not scanning
     // through all tiles.
     if (!exact) {
-      final PriorityQueue<Pair<OptiqSchema.TableEntry, TileKey>> queue =
-          new PriorityQueue<Pair<OptiqSchema.TableEntry, TileKey>>(1, C);
+      final PriorityQueue<Pair<CalciteSchema.TableEntry, TileKey>> queue =
+          new PriorityQueue<Pair<CalciteSchema.TableEntry, TileKey>>(1, C);
       for (Map.Entry<TileKey, MaterializationKey> entry
           : actor.keyByTile.entrySet()) {
         final TileKey tileKey2 = entry.getKey();
@@ -242,7 +255,7 @@ public class MaterializationService {
             && !tileKey2.dimensions.equals(groupSet)
             && allSatisfiable(measureList, tileKey2)) {
           materializationKey = entry.getValue();
-          final OptiqSchema.TableEntry tableEntry =
+          final CalciteSchema.TableEntry tableEntry =
               checkValid(materializationKey);
           if (tableEntry != null) {
             queue.add(Pair.of(tableEntry, tileKey2));
@@ -250,8 +263,8 @@ public class MaterializationService {
         }
       }
       if (!queue.isEmpty()) {
-        final Pair<OptiqSchema.TableEntry, TileKey> best = queue.peek();
-        for (Pair<OptiqSchema.TableEntry, TileKey> pair : queue) {
+        final Pair<CalciteSchema.TableEntry, TileKey> best = queue.peek();
+        for (Pair<CalciteSchema.TableEntry, TileKey> pair : queue) {
           System.out.println("table=" + pair.left.path() + " "
               + pair.left.getTable().getStatistic().getRowCount());
         }
@@ -285,7 +298,8 @@ public class MaterializationService {
         defineMaterialization(schema, newTileKey, sql, schema.path(null),
             "m" + groupSet, true);
     if (materializationKey != null) {
-      final OptiqSchema.TableEntry tableEntry = checkValid(materializationKey);
+      final CalciteSchema.TableEntry tableEntry =
+          checkValid(materializationKey);
       if (tableEntry != null) {
         // Obsolete all of the narrower tiles.
         for (TileKey tileKey1 : obsolete) {
@@ -318,7 +332,7 @@ public class MaterializationService {
    * schema. (Each root schema defines a disconnected namespace, with no overlap
    * with the current schema. Especially in a test run, the contents of two
    * root schemas may look similar.) */
-  public List<Prepare.Materialization> query(OptiqSchema rootSchema) {
+  public List<Prepare.Materialization> query(CalciteSchema rootSchema) {
     final List<Prepare.Materialization> list =
         new ArrayList<Prepare.Materialization>();
     for (MaterializationActor.Materialization materialization

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/materialize/TileKey.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/materialize/TileKey.java b/core/src/main/java/org/apache/calcite/materialize/TileKey.java
index 1ba6136..642edb6 100644
--- a/core/src/main/java/org/apache/calcite/materialize/TileKey.java
+++ b/core/src/main/java/org/apache/calcite/materialize/TileKey.java
@@ -14,10 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.materialize;
-
-import org.eigenbase.util.Util;
+package org.apache.calcite.materialize;
 
+import com.google.common.base.Objects;
 import com.google.common.collect.ImmutableList;
 
 import java.util.BitSet;
@@ -26,7 +25,7 @@ import java.util.BitSet;
  * lattice that is the basis of a materialization.
  *
  * <p>Holds similar information to a
- * {@link net.hydromatic.optiq.materialize.Lattice.Tile} but a lattice is
+ * {@link org.apache.calcite.materialize.Lattice.Tile} but a lattice is
  * immutable and tiles are not added after their creation. */
 public class TileKey {
   public final Lattice lattice;
@@ -42,7 +41,7 @@ public class TileKey {
   }
 
   @Override public int hashCode() {
-    return Util.hashV(lattice, dimensions);
+    return Objects.hashCode(lattice, dimensions);
   }
 
   @Override public boolean equals(Object obj) {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/materialize/TileSuggester.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/materialize/TileSuggester.java b/core/src/main/java/org/apache/calcite/materialize/TileSuggester.java
index 74ffe27..65095cf 100644
--- a/core/src/main/java/org/apache/calcite/materialize/TileSuggester.java
+++ b/core/src/main/java/org/apache/calcite/materialize/TileSuggester.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.materialize;
+package org.apache.calcite.materialize;
 
 import com.google.common.base.Function;
 import com.google.common.collect.ImmutableList;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/materialize/package-info.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/materialize/package-info.java b/core/src/main/java/org/apache/calcite/materialize/package-info.java
index ebeda47..0aab1ca 100644
--- a/core/src/main/java/org/apache/calcite/materialize/package-info.java
+++ b/core/src/main/java/org/apache/calcite/materialize/package-info.java
@@ -18,10 +18,10 @@
 /**
  * Management of materialized query results.
  *
- * <p>An actor ({@link net.hydromatic.optiq.materialize.MaterializationActor})
+ * <p>An actor ({@link org.apache.calcite.materialize.MaterializationActor})
  * maintains the state of all
  * materializations in the system and is wrapped in a service
- * ({@link net.hydromatic.optiq.materialize.MaterializationService})
+ * ({@link org.apache.calcite.materialize.MaterializationService})
  * for access from other parts of the system.</p>
  *
  * <p>Optimizer rules allow Calcite to rewrite queries using materializations,
@@ -32,6 +32,6 @@
  * instantiating materializations from the intermediate results of queries, and
  * recognize what materializations would be useful based on actual query load.
  */
-package net.hydromatic.optiq.materialize;
+package org.apache.calcite.materialize;
 
 // End package-info.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/model/JsonColumn.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/model/JsonColumn.java b/core/src/main/java/org/apache/calcite/model/JsonColumn.java
index d6c08dd..2b9a361 100644
--- a/core/src/main/java/org/apache/calcite/model/JsonColumn.java
+++ b/core/src/main/java/org/apache/calcite/model/JsonColumn.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.model;
+package org.apache.calcite.model;
 
 /**
  * JSON object representing a column.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/model/JsonCustomSchema.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/model/JsonCustomSchema.java b/core/src/main/java/org/apache/calcite/model/JsonCustomSchema.java
index 15ebd76..3144d38 100644
--- a/core/src/main/java/org/apache/calcite/model/JsonCustomSchema.java
+++ b/core/src/main/java/org/apache/calcite/model/JsonCustomSchema.java
@@ -14,18 +14,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.model;
+package org.apache.calcite.model;
 
 import java.util.Map;
 
 /**
  * JSON schema element that represents a custom schema.
  *
- * @see net.hydromatic.optiq.model.JsonRoot Description of schema elements
+ * @see org.apache.calcite.model.JsonRoot Description of schema elements
  */
 public class JsonCustomSchema extends JsonMapSchema {
   /** Name of the factory class for this schema. Must implement interface
-   * {@link net.hydromatic.optiq.SchemaFactory} and have a public default
+   * {@link org.apache.calcite.schema.SchemaFactory} and have a public default
    * constructor. */
   public String factory;
 
@@ -36,8 +36,7 @@ public class JsonCustomSchema extends JsonMapSchema {
     handler.visit(this);
   }
 
-  @Override
-  public String toString() {
+  @Override public String toString() {
     return "JsonCustomSchema(name=" + name + ")";
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/model/JsonCustomTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/model/JsonCustomTable.java b/core/src/main/java/org/apache/calcite/model/JsonCustomTable.java
index db699a6..07a73d9 100644
--- a/core/src/main/java/org/apache/calcite/model/JsonCustomTable.java
+++ b/core/src/main/java/org/apache/calcite/model/JsonCustomTable.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.model;
+package org.apache.calcite.model;
 
 import java.util.Map;
 
@@ -25,7 +25,7 @@ import java.util.Map;
  */
 public class JsonCustomTable extends JsonTable {
   /** Name of the factory class for this table. Must implement interface
-   * {@link net.hydromatic.optiq.TableFactory} and have a public default
+   * {@link org.apache.calcite.schema.TableFactory} and have a public default
    * constructor. */
   public String factory;
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/model/JsonFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/model/JsonFunction.java b/core/src/main/java/org/apache/calcite/model/JsonFunction.java
index 5e290b3..5c24ec9 100644
--- a/core/src/main/java/org/apache/calcite/model/JsonFunction.java
+++ b/core/src/main/java/org/apache/calcite/model/JsonFunction.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.model;
+package org.apache.calcite.model;
 
 import java.util.List;
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/model/JsonJdbcSchema.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/model/JsonJdbcSchema.java b/core/src/main/java/org/apache/calcite/model/JsonJdbcSchema.java
index eb2e1b6..61923d9 100644
--- a/core/src/main/java/org/apache/calcite/model/JsonJdbcSchema.java
+++ b/core/src/main/java/org/apache/calcite/model/JsonJdbcSchema.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.model;
+package org.apache.calcite.model;
 
 /**
  * JSON object representing a schema that maps to a JDBC database.
@@ -29,8 +29,7 @@ public class JsonJdbcSchema extends JsonSchema {
   public String jdbcCatalog;
   public String jdbcSchema;
 
-  @Override
-  public void accept(ModelHandler handler) {
+  @Override public void accept(ModelHandler handler) {
     handler.visit(this);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/model/JsonLattice.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/model/JsonLattice.java b/core/src/main/java/org/apache/calcite/model/JsonLattice.java
index ec8578d..80974a3 100644
--- a/core/src/main/java/org/apache/calcite/model/JsonLattice.java
+++ b/core/src/main/java/org/apache/calcite/model/JsonLattice.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.model;
+package org.apache.calcite.model;
 
 import com.google.common.collect.Lists;
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/model/JsonMapSchema.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/model/JsonMapSchema.java b/core/src/main/java/org/apache/calcite/model/JsonMapSchema.java
index 51499cc..e65a211 100644
--- a/core/src/main/java/org/apache/calcite/model/JsonMapSchema.java
+++ b/core/src/main/java/org/apache/calcite/model/JsonMapSchema.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.model;
+package org.apache.calcite.model;
 
 import java.util.ArrayList;
 import java.util.List;
@@ -28,13 +28,11 @@ public class JsonMapSchema extends JsonSchema {
   public final List<JsonTable> tables = new ArrayList<JsonTable>();
   public final List<JsonFunction> functions = new ArrayList<JsonFunction>();
 
-  @Override
-  public void accept(ModelHandler handler) {
+  @Override public void accept(ModelHandler handler) {
     handler.visit(this);
   }
 
-  @Override
-  public void visitChildren(ModelHandler modelHandler) {
+  @Override public void visitChildren(ModelHandler modelHandler) {
     super.visitChildren(modelHandler);
     for (JsonTable jsonTable : tables) {
       jsonTable.accept(modelHandler);

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/model/JsonMaterialization.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/model/JsonMaterialization.java b/core/src/main/java/org/apache/calcite/model/JsonMaterialization.java
index 17d08fa..59b4828 100644
--- a/core/src/main/java/org/apache/calcite/model/JsonMaterialization.java
+++ b/core/src/main/java/org/apache/calcite/model/JsonMaterialization.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.model;
+package org.apache.calcite.model;
 
 /**
  * Element that describes how a table is a materialization of a query.
@@ -30,8 +30,7 @@ public class JsonMaterialization {
     handler.visit(this);
   }
 
-  @Override
-  public String toString() {
+  @Override public String toString() {
     return "JsonMaterialization(table=" + table + ", view=" + view + ")";
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/model/JsonMeasure.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/model/JsonMeasure.java b/core/src/main/java/org/apache/calcite/model/JsonMeasure.java
index 3499b59..e813c22 100644
--- a/core/src/main/java/org/apache/calcite/model/JsonMeasure.java
+++ b/core/src/main/java/org/apache/calcite/model/JsonMeasure.java
@@ -14,14 +14,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.model;
+package org.apache.calcite.model;
 
 /**
  * An aggregate function applied to a column (or columns) of a lattice.
  *
- * <p>Occurs in a {@link net.hydromatic.optiq.model.JsonTile},
+ * <p>Occurs in a {@link org.apache.calcite.model.JsonTile},
  * and there is a default list in
- * {@link net.hydromatic.optiq.model.JsonLattice}.
+ * {@link org.apache.calcite.model.JsonLattice}.
  *
  * @see JsonRoot Description of schema elements
  */

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/model/JsonRoot.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/model/JsonRoot.java b/core/src/main/java/org/apache/calcite/model/JsonRoot.java
index 52ae471..25e1fa3 100644
--- a/core/src/main/java/org/apache/calcite/model/JsonRoot.java
+++ b/core/src/main/java/org/apache/calcite/model/JsonRoot.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.model;
+package org.apache.calcite.model;
 
 import java.util.ArrayList;
 import java.util.List;
@@ -33,7 +33,7 @@ import java.util.List;
  *     {@link JsonTable} (in collection {@link JsonMapSchema#tables tables})
  *       {@link JsonColumn} (in collection {@link JsonTable#columns columns}
  *     {@link JsonView}
- *     {@link JsonFunction}  (in collection {@link JsonMapSchema#functions functions})
+ *     {@link JsonFunction} (in collection {@link JsonMapSchema#functions functions})
  *     {@link JsonLattice} (in collection {@link JsonSchema#lattices lattices})
  *       {@link JsonMeasure} (in collection {@link JsonLattice#defaultMeasures defaultMeasures})
  *       {@link JsonTile} (in collection {@link JsonLattice#tiles tiles})

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/model/JsonSchema.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/model/JsonSchema.java b/core/src/main/java/org/apache/calcite/model/JsonSchema.java
index 0eb2e29..07e326f 100644
--- a/core/src/main/java/org/apache/calcite/model/JsonSchema.java
+++ b/core/src/main/java/org/apache/calcite/model/JsonSchema.java
@@ -14,11 +14,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.model;
+package org.apache.calcite.model;
 
 import com.fasterxml.jackson.annotation.JsonSubTypes;
 import com.fasterxml.jackson.annotation.JsonTypeInfo;
-
 import com.google.common.collect.Lists;
 
 import java.util.List;
@@ -58,10 +57,10 @@ public abstract class JsonSchema {
    * <p>If {@code true}, Calcite will cache the metadata the first time it reads
    * it. This can lead to better performance, especially if name-matching is
    * case-insensitive
-   * (see {@link net.hydromatic.optiq.config.Lex#caseSensitive}).
+   * (see {@link org.apache.calcite.config.Lex#caseSensitive}).
    * However, it also leads to the problem of cache staleness.
    * A particular schema implementation can override the
-   * {@link net.hydromatic.optiq.Schema#contentsHaveChangedSince(long, long)}
+   * {@link org.apache.calcite.schema.Schema#contentsHaveChangedSince(long, long)}
    * method to tell Calcite when it should consider its cache to be out of
    * date.</p>
    *

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/model/JsonTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/model/JsonTable.java b/core/src/main/java/org/apache/calcite/model/JsonTable.java
index 0c82f91..c0e7d8d 100644
--- a/core/src/main/java/org/apache/calcite/model/JsonTable.java
+++ b/core/src/main/java/org/apache/calcite/model/JsonTable.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.model;
+package org.apache.calcite.model;
 
 import com.fasterxml.jackson.annotation.JsonSubTypes;
 import com.fasterxml.jackson.annotation.JsonTypeInfo;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/model/JsonTile.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/model/JsonTile.java b/core/src/main/java/org/apache/calcite/model/JsonTile.java
index f7159f9..aef1d2e 100644
--- a/core/src/main/java/org/apache/calcite/model/JsonTile.java
+++ b/core/src/main/java/org/apache/calcite/model/JsonTile.java
@@ -14,14 +14,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.model;
+package org.apache.calcite.model;
 
 import com.google.common.collect.Lists;
 
 import java.util.List;
 
 /**
- * Materialized view within a {@link net.hydromatic.optiq.model.JsonLattice}.
+ * Materialized view within a {@link org.apache.calcite.model.JsonLattice}.
  *
  * <p>A tile is defined in terms of its dimensionality (the grouping columns,
  * drawn from the lattice) and measures (aggregate functions applied to

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/model/JsonView.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/model/JsonView.java b/core/src/main/java/org/apache/calcite/model/JsonView.java
index 4ba9dca..f482765 100644
--- a/core/src/main/java/org/apache/calcite/model/JsonView.java
+++ b/core/src/main/java/org/apache/calcite/model/JsonView.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.model;
+package org.apache.calcite.model;
 
 import java.util.List;
 
@@ -35,8 +35,7 @@ public class JsonView extends JsonTable {
     handler.visit(this);
   }
 
-  @Override
-  public String toString() {
+  @Override public String toString() {
     return "JsonView(name=" + name + ")";
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/model/ModelHandler.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/model/ModelHandler.java b/core/src/main/java/org/apache/calcite/model/ModelHandler.java
index 94c7a96..20a4596 100644
--- a/core/src/main/java/org/apache/calcite/model/ModelHandler.java
+++ b/core/src/main/java/org/apache/calcite/model/ModelHandler.java
@@ -14,44 +14,61 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package net.hydromatic.optiq.model;
-
-import net.hydromatic.optiq.*;
-import net.hydromatic.optiq.impl.*;
-import net.hydromatic.optiq.impl.jdbc.JdbcSchema;
-import net.hydromatic.optiq.jdbc.OptiqConnection;
-import net.hydromatic.optiq.jdbc.OptiqSchema;
-import net.hydromatic.optiq.materialize.Lattice;
-
-import org.eigenbase.util.Pair;
-import org.eigenbase.util.Util;
+package org.apache.calcite.model;
+
+import org.apache.calcite.adapter.jdbc.JdbcSchema;
+import org.apache.calcite.jdbc.CalciteConnection;
+import org.apache.calcite.jdbc.CalciteSchema;
+import org.apache.calcite.materialize.Lattice;
+import org.apache.calcite.schema.AggregateFunction;
+import org.apache.calcite.schema.ScalarFunction;
+import org.apache.calcite.schema.Schema;
+import org.apache.calcite.schema.SchemaFactory;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.schema.Table;
+import org.apache.calcite.schema.TableFactory;
+import org.apache.calcite.schema.TableFunction;
+import org.apache.calcite.schema.TableMacro;
+import org.apache.calcite.schema.impl.AbstractSchema;
+import org.apache.calcite.schema.impl.AggregateFunctionImpl;
+import org.apache.calcite.schema.impl.MaterializedViewTable;
+import org.apache.calcite.schema.impl.ScalarFunctionImpl;
+import org.apache.calcite.schema.impl.TableFunctionImpl;
+import org.apache.calcite.schema.impl.TableMacroImpl;
+import org.apache.calcite.schema.impl.ViewTable;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
 
 import com.fasterxml.jackson.core.JsonParser;
 import com.fasterxml.jackson.databind.ObjectMapper;
-
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 
 import java.io.File;
 import java.io.IOException;
 import java.sql.SQLException;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
 import javax.sql.DataSource;
 
-import static org.eigenbase.util.Stacks.*;
+import static org.apache.calcite.util.Stacks.peek;
+import static org.apache.calcite.util.Stacks.pop;
+import static org.apache.calcite.util.Stacks.push;
 
 /**
  * Reads a model and creates schema objects accordingly.
  */
 public class ModelHandler {
-  private final OptiqConnection connection;
+  private final CalciteConnection connection;
   private final List<Pair<String, SchemaPlus>> schemaStack =
       new ArrayList<Pair<String, SchemaPlus>>();
   private final String modelUri;
   Lattice.Builder latticeBuilder;
   Lattice.TileBuilder tileBuilder;
 
-  public ModelHandler(OptiqConnection connection, String uri)
+  public ModelHandler(CalciteConnection connection, String uri)
       throws IOException {
     super();
     this.connection = connection;
@@ -194,8 +211,8 @@ public class ModelHandler {
       final Schema schema =
           schemaFactory.create(
               parentSchema, jsonSchema.name, operandMap(jsonSchema.operand));
-      final SchemaPlus optiqSchema = parentSchema.add(jsonSchema.name, schema);
-      populateSchema(jsonSchema, optiqSchema);
+      final SchemaPlus schemaPlus = parentSchema.add(jsonSchema.name, schema);
+      populateSchema(jsonSchema, schemaPlus);
     } catch (Exception e) {
       throw new RuntimeException("Error instantiating " + jsonSchema, e);
     }
@@ -232,8 +249,8 @@ public class ModelHandler {
     JdbcSchema schema =
         JdbcSchema.create(parentSchema, jsonSchema.name, dataSource,
             jsonSchema.jdbcCatalog, jsonSchema.jdbcSchema);
-    final SchemaPlus optiqSchema = parentSchema.add(jsonSchema.name, schema);
-    populateSchema(jsonSchema, optiqSchema);
+    final SchemaPlus schemaPlus = parentSchema.add(jsonSchema.name, schema);
+    populateSchema(jsonSchema, schemaPlus);
   }
 
   public void visit(JsonMaterialization jsonMaterialization) {
@@ -242,12 +259,12 @@ public class ModelHandler {
       if (!schema.isMutable()) {
         throw new RuntimeException(
             "Cannot define materialization; parent schema '"
-            + currentSchemaName()
-            + "' is not a SemiMutableSchema");
+                + currentSchemaName()
+                + "' is not a SemiMutableSchema");
       }
-      OptiqSchema optiqSchema = OptiqSchema.from(schema);
+      CalciteSchema calciteSchema = CalciteSchema.from(schema);
       schema.add(jsonMaterialization.view,
-          MaterializedViewTable.create(optiqSchema,
+          MaterializedViewTable.create(calciteSchema,
               jsonMaterialization.getSql(), null, jsonMaterialization.table));
     } catch (Exception e) {
       throw new RuntimeException("Error instantiating " + jsonMaterialization,
@@ -259,14 +276,13 @@ public class ModelHandler {
     try {
       final SchemaPlus schema = currentSchema();
       if (!schema.isMutable()) {
-        throw new RuntimeException(
-            "Cannot define lattice; parent schema '"
+        throw new RuntimeException("Cannot define lattice; parent schema '"
             + currentSchemaName()
             + "' is not a SemiMutableSchema");
       }
-      OptiqSchema optiqSchema = OptiqSchema.from(schema);
+      CalciteSchema calciteSchema = CalciteSchema.from(schema);
       Lattice.Builder latticeBuilder =
-          Lattice.builder(optiqSchema, jsonLattice.getSql())
+          Lattice.builder(calciteSchema, jsonLattice.getSql())
               .auto(jsonLattice.auto)
               .algorithm(jsonLattice.algorithm);
       if (jsonLattice.rowCountEstimate != null) {
@@ -333,9 +349,8 @@ public class ModelHandler {
   private SchemaPlus currentMutableSchema(String elementType) {
     final SchemaPlus schema = currentSchema();
     if (!schema.isMutable()) {
-      throw new RuntimeException(
-          "Cannot define " + elementType + "; parent schema '"
-          + schema.getName() + "' is not mutable");
+      throw new RuntimeException("Cannot define " + elementType
+          + "; parent schema '" + schema.getName() + "' is not mutable");
     }
     return schema;
   }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/model/package-info.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/model/package-info.java b/core/src/main/java/org/apache/calcite/model/package-info.java
index c44d6fe..38a08a5 100644
--- a/core/src/main/java/org/apache/calcite/model/package-info.java
+++ b/core/src/main/java/org/apache/calcite/model/package-info.java
@@ -21,19 +21,19 @@
  * <p>Models are specified using a <code>model=&lt;uri&gt;</code> parameter on
  * the JDBC connect string. Calcite loads the model while initializing the
  * connection. It first parses the JSON, then uses a
- * {@link net.hydromatic.optiq.model.ModelHandler} as visitor over the parse
+ * {@link org.apache.calcite.model.ModelHandler} as visitor over the parse
  * tree.</p>
  *
  * <p>There are standard implementations of schema and table, but the user can
  * provide their own by implementing the
- * {@link net.hydromatic.optiq.SchemaFactory}
- * or {@link net.hydromatic.optiq.TableFactory}
+ * {@link org.apache.calcite.schema.SchemaFactory}
+ * or {@link org.apache.calcite.schema.TableFactory}
  * interfaces and including a custom schema in the model.</p>
  *
  * <p>There are several examples of schemas in the
  * <a href="https://github.com/julianhyde/blog/master/optiq-csv/TUTORIAL.md">optiq-csv
  * tutorial</a>.
  */
-package net.hydromatic.optiq.model;
+package org.apache.calcite.model;
 
 // End package-info.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/package-info.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/package-info.java b/core/src/main/java/org/apache/calcite/package-info.java
index 0afdc3f..87ef21f 100644
--- a/core/src/main/java/org/apache/calcite/package-info.java
+++ b/core/src/main/java/org/apache/calcite/package-info.java
@@ -18,6 +18,6 @@
 /**
  * Main package for Calcite, the dynamic data management platform.
  */
-package net.hydromatic.optiq;
+package org.apache.calcite;
 
 // End package-info.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/plan/AbstractRelOptPlanner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/AbstractRelOptPlanner.java b/core/src/main/java/org/apache/calcite/plan/AbstractRelOptPlanner.java
index 9823b74..b25c28c 100644
--- a/core/src/main/java/org/apache/calcite/plan/AbstractRelOptPlanner.java
+++ b/core/src/main/java/org/apache/calcite/plan/AbstractRelOptPlanner.java
@@ -14,22 +14,29 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.relopt;
+package org.apache.calcite.plan;
 
-import java.util.*;
-import java.util.logging.*;
-import java.util.regex.*;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.rel.metadata.*;
-import org.eigenbase.relopt.volcano.RelSubset;
-import org.eigenbase.util.*;
-
-import static org.eigenbase.util.Static.RESOURCE;
+import org.apache.calcite.plan.volcano.RelSubset;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.metadata.RelMetadataProvider;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.util.CancelFlag;
 
 import com.google.common.base.Predicate;
 import com.google.common.collect.Iterables;
 
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.logging.Level;
+import java.util.regex.Pattern;
+
+import static org.apache.calcite.util.Static.RESOURCE;
+
 
 /**
  * Abstract base for implementations of the {@link RelOptPlanner} interface.
@@ -146,8 +153,7 @@ public abstract class AbstractRelOptPlanner implements RelOptPlanner {
         // This rule has the same description as one previously
         // registered, yet it is not equal. You may need to fix the
         // rule's equals and hashCode methods.
-        throw new AssertionError(
-            "Rule's description should be unique; "
+        throw new AssertionError("Rule's description should be unique; "
             + "existing rule=" + existingRule + "; new rule=" + rule);
       }
     }
@@ -293,8 +299,7 @@ public abstract class AbstractRelOptPlanner implements RelOptPlanner {
     assert ruleCall.getRule().matches(ruleCall);
     if (isRuleExcluded(ruleCall.getRule())) {
       if (LOGGER.isLoggable(Level.FINE)) {
-        LOGGER.fine(
-            "call#" + ruleCall.id
+        LOGGER.fine("call#" + ruleCall.id
             + ": Rule [" + ruleCall.getRule() + "] not fired"
             + " due to exclusion filter");
       }
@@ -344,8 +349,7 @@ public abstract class AbstractRelOptPlanner implements RelOptPlanner {
       RelNode newRel,
       boolean before) {
     if (before && LOGGER.isLoggable(Level.FINE)) {
-      LOGGER.fine(
-          "call#" + ruleCall.id
+      LOGGER.fine("call#" + ruleCall.id
           + ": Rule " + ruleCall.getRule() + " arguments "
           + Arrays.toString(ruleCall.rels) + " produced "
           + newRel);

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/plan/CommonRelSubExprRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/CommonRelSubExprRule.java b/core/src/main/java/org/apache/calcite/plan/CommonRelSubExprRule.java
index 9133ed2..7886ad2 100644
--- a/core/src/main/java/org/apache/calcite/plan/CommonRelSubExprRule.java
+++ b/core/src/main/java/org/apache/calcite/plan/CommonRelSubExprRule.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.relopt;
+package org.apache.calcite.plan;
 
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/plan/Context.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/Context.java b/core/src/main/java/org/apache/calcite/plan/Context.java
index 093cf52..3537d98 100644
--- a/core/src/main/java/org/apache/calcite/plan/Context.java
+++ b/core/src/main/java/org/apache/calcite/plan/Context.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.relopt;
+package org.apache.calcite.plan;
 
 /**
  * Provides library users a way to store data within the planner session and
@@ -31,3 +31,5 @@ public interface Context {
   <T> T unwrap(Class<T> clazz);
 
 }
+
+// End Context.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/plan/Contexts.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/Contexts.java b/core/src/main/java/org/apache/calcite/plan/Contexts.java
index 54dddaa..e2fa408 100644
--- a/core/src/main/java/org/apache/calcite/plan/Contexts.java
+++ b/core/src/main/java/org/apache/calcite/plan/Contexts.java
@@ -14,9 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.relopt;
+package org.apache.calcite.plan;
 
-import net.hydromatic.optiq.config.OptiqConnectionConfig;
+import org.apache.calcite.config.CalciteConnectionConfig;
 
 /**
  * Utilities for {@link Context}.
@@ -27,8 +27,8 @@ public class Contexts {
   private Contexts() {}
 
   /** Returns a context that contains a
-   * {@link net.hydromatic.optiq.config.OptiqConnectionConfig}. */
-  public static Context withConfig(OptiqConnectionConfig config) {
+   * {@link org.apache.calcite.config.CalciteConnectionConfig}. */
+  public static Context withConfig(CalciteConnectionConfig config) {
     return new ConfigContext(config);
   }
 
@@ -38,11 +38,11 @@ public class Contexts {
   }
 
   /** Context that contains a
-   * {@link net.hydromatic.optiq.config.OptiqConnectionConfig}. */
+   * {@link org.apache.calcite.config.CalciteConnectionConfig}. */
   private static class ConfigContext implements Context {
-    private OptiqConnectionConfig config;
+    private CalciteConnectionConfig config;
 
-    public ConfigContext(OptiqConnectionConfig config) {
+    public ConfigContext(CalciteConnectionConfig config) {
       this.config = config;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/plan/Convention.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/Convention.java b/core/src/main/java/org/apache/calcite/plan/Convention.java
index d1894d3..dbd1a5c 100644
--- a/core/src/main/java/org/apache/calcite/plan/Convention.java
+++ b/core/src/main/java/org/apache/calcite/plan/Convention.java
@@ -14,9 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.relopt;
+package org.apache.calcite.plan;
 
-import org.eigenbase.rel.RelNode;
+import org.apache.calcite.rel.RelNode;
 
 /**
  * Calling convention trait.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 514a2f5..9de4762 100644
--- a/core/src/main/java/org/apache/calcite/plan/ConventionTraitDef.java
+++ b/core/src/main/java/org/apache/calcite/plan/ConventionTraitDef.java
@@ -14,20 +14,23 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.relopt;
+package org.apache.calcite.plan;
 
-import java.util.*;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.rel.convert.*;
-import org.eigenbase.util.Pair;
-import org.eigenbase.util.Util;
-
-import net.hydromatic.optiq.util.graph.*;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+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.Graphs;
 
 import com.google.common.collect.HashMultimap;
 import com.google.common.collect.Multimap;
 
+import java.util.List;
+import java.util.WeakHashMap;
+
 /**
  * Definition of the the convention trait.
  * A new set of conversion information is created for
@@ -146,8 +149,7 @@ public class ConventionTraitDef extends RelTraitDef<Convention> {
                   converted, previous, arc,
                   conversionData.mapArcToConverterRule);
           if (converted == null) {
-            throw Util.newInternal(
-                "Converter from " + previous
+            throw Util.newInternal("Converter from " + previous
                 + " to " + arc
                 + " guaranteed that it could convert any relexp");
           }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/plan/MulticastRelOptListener.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/MulticastRelOptListener.java b/core/src/main/java/org/apache/calcite/plan/MulticastRelOptListener.java
index 549e44f..9b2bcc0 100644
--- a/core/src/main/java/org/apache/calcite/plan/MulticastRelOptListener.java
+++ b/core/src/main/java/org/apache/calcite/plan/MulticastRelOptListener.java
@@ -14,9 +14,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.relopt;
+package org.apache.calcite.plan;
 
-import java.util.*;
+import java.util.ArrayList;
+import java.util.List;
 
 /**
  * MulticastRelOptListener implements the {@link RelOptListener} interface by

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/plan/RelImplementor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/RelImplementor.java b/core/src/main/java/org/apache/calcite/plan/RelImplementor.java
index ad41d88..6ae9d6c 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelImplementor.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelImplementor.java
@@ -14,15 +14,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.relopt;
+package org.apache.calcite.plan;
 
-import org.eigenbase.rel.*;
+import org.apache.calcite.rel.RelNode;
 
 /**
- * Callback used to hold state while converting a tree of {@link RelNode
- * relational expressions} into a plan. Calling conventions typically have their
- * own protocol for walking over a tree, and correspondingly have their own
- * implementors, which are subclasses of <code>RelImplementor</code>.
+ * Callback used to hold state while converting a tree of
+ * {@link RelNode relational expressions} into a plan. Calling
+ * conventions typically have their own protocol for walking over a
+ * tree, and correspondingly have their own implementors, which are
+ * subclasses of <code>RelImplementor</code>.
  */
 public interface RelImplementor {
   //~ Methods ----------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/plan/RelOptAbstractTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/RelOptAbstractTable.java b/core/src/main/java/org/apache/calcite/plan/RelOptAbstractTable.java
index 9efe276..5496477 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptAbstractTable.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptAbstractTable.java
@@ -14,17 +14,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.relopt;
+package org.apache.calcite.plan;
 
-import java.util.*;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.reltype.*;
-
-import net.hydromatic.linq4j.expressions.Expression;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.logical.LogicalTableScan;
+import org.apache.calcite.rel.type.RelDataType;
 
 import com.google.common.collect.ImmutableList;
 
+import java.util.BitSet;
+import java.util.Collections;
+import java.util.List;
+
 /**
  * Partial implementation of {@link RelOptTable}.
  */
@@ -85,7 +88,7 @@ public abstract class RelOptAbstractTable implements RelOptTable {
   }
 
   public RelNode toRel(ToRelContext context) {
-    return new TableAccessRel(context.getCluster(), this);
+    return new LogicalTableScan(context.getCluster(), this);
   }
 
   public Expression getExpression(Class clazz) {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/plan/RelOptCluster.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/RelOptCluster.java b/core/src/main/java/org/apache/calcite/plan/RelOptCluster.java
index 30688a6..d1171ab 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptCluster.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptCluster.java
@@ -14,19 +14,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.relopt;
+package org.apache.calcite.plan;
 
-import org.eigenbase.rel.*;
-import org.eigenbase.rel.metadata.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.rex.*;
+import org.apache.calcite.rel.metadata.DefaultRelMetadataProvider;
+import org.apache.calcite.rel.metadata.MetadataFactory;
+import org.apache.calcite.rel.metadata.MetadataFactoryImpl;
+import org.apache.calcite.rel.metadata.RelMetadataProvider;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexNode;
 
 /**
- * A <code>RelOptCluster</code> is a collection of {@link RelNode relational
- * expressions} which have the same environment.
- *
- * <p>See the comment against <code>net.sf.saffron.oj.xlat.QueryInfo</code> on
- * why you should put fields in that class, not this one.</p>
+ * An environment for related relational expressions during the
+ * optimization of a query.
  */
 public class RelOptCluster {
   //~ Instance fields --------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/plan/RelOptConnection.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/RelOptConnection.java b/core/src/main/java/org/apache/calcite/plan/RelOptConnection.java
index 39b1800..0005e62 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptConnection.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptConnection.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.relopt;
+package org.apache.calcite.plan;
 
 /**
  * The planner's view of a connection to a database.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/plan/RelOptCost.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/RelOptCost.java b/core/src/main/java/org/apache/calcite/plan/RelOptCost.java
index 50ab422..1e19a72 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptCost.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptCost.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.relopt;
+package org.apache.calcite.plan;
 
 /**
  * RelOptCost defines an interface for optimizer cost in terms of number of rows
@@ -31,8 +31,8 @@ public interface RelOptCost {
 
   /**
    * @return number of rows processed; this should not be confused with the
-   * row count produced by a relational expression ({@link
-   * org.eigenbase.rel.RelNode#getRows})
+   * row count produced by a relational expression
+   * ({@link org.apache.calcite.rel.RelNode#getRows})
    */
   double getRows();
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/plan/RelOptCostFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/RelOptCostFactory.java b/core/src/main/java/org/apache/calcite/plan/RelOptCostFactory.java
index 9486dc3..cb3ec0b 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptCostFactory.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptCostFactory.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.relopt;
+package org.apache.calcite.plan;
 
 /**
  * Cost model for query planning.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/plan/RelOptCostImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/RelOptCostImpl.java b/core/src/main/java/org/apache/calcite/plan/RelOptCostImpl.java
index 81dded2..7c5aeca 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptCostImpl.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptCostImpl.java
@@ -14,9 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.relopt;
+package org.apache.calcite.plan;
 
-import org.eigenbase.util.Util;
+import org.apache.calcite.util.Util;
 
 /**
  * RelOptCostImpl provides a default implementation for the {@link RelOptCost}
@@ -69,8 +69,7 @@ public class RelOptCostImpl implements RelOptCost {
     return getRows() < other.getRows();
   }
 
-  @Override
-  public int hashCode() {
+  @Override public int hashCode() {
     return Util.hashCode(getRows());
   }
 


[54/58] [abbrv] incubator-calcite git commit: [CALCITE-460] Add ImmutableBitSet and replace uses of BitSet

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/core/src/main/java/org/apache/calcite/rel/metadata/RelMdPopulationSize.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdPopulationSize.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdPopulationSize.java
index 2afbc80..2009d09 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdPopulationSize.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdPopulationSize.java
@@ -26,10 +26,9 @@ import org.apache.calcite.rel.core.Sort;
 import org.apache.calcite.rel.core.Union;
 import org.apache.calcite.rel.core.Values;
 import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.util.BitSets;
 import org.apache.calcite.util.BuiltInMethod;
+import org.apache.calcite.util.ImmutableBitSet;
 
-import java.util.BitSet;
 import java.util.List;
 
 /**
@@ -47,19 +46,19 @@ public class RelMdPopulationSize {
 
   //~ Methods ----------------------------------------------------------------
 
-  public Double getPopulationSize(Filter rel, BitSet groupKey) {
+  public Double getPopulationSize(Filter rel, ImmutableBitSet groupKey) {
     return RelMetadataQuery.getPopulationSize(
         rel.getInput(),
         groupKey);
   }
 
-  public Double getPopulationSize(Sort rel, BitSet groupKey) {
+  public Double getPopulationSize(Sort rel, ImmutableBitSet groupKey) {
     return RelMetadataQuery.getPopulationSize(
         rel.getInput(),
         groupKey);
   }
 
-  public Double getPopulationSize(Union rel, BitSet groupKey) {
+  public Double getPopulationSize(Union rel, ImmutableBitSet groupKey) {
     Double population = 0.0;
     for (RelNode input : rel.getInputs()) {
       Double subPop = RelMetadataQuery.getPopulationSize(input, groupKey);
@@ -71,39 +70,35 @@ public class RelMdPopulationSize {
     return population;
   }
 
-  public Double getPopulationSize(Join rel, BitSet groupKey) {
+  public Double getPopulationSize(Join rel, ImmutableBitSet groupKey) {
     return RelMdUtil.getJoinPopulationSize(rel, groupKey);
   }
 
-  public Double getPopulationSize(SemiJoin rel, BitSet groupKey) {
-    return RelMetadataQuery.getPopulationSize(
-        rel.getLeft(),
-        groupKey);
+  public Double getPopulationSize(SemiJoin rel, ImmutableBitSet groupKey) {
+    return RelMetadataQuery.getPopulationSize(rel.getLeft(), groupKey);
   }
 
-  public Double getPopulationSize(Aggregate rel, BitSet groupKey) {
-    BitSet childKey = new BitSet();
+  public Double getPopulationSize(Aggregate rel, ImmutableBitSet groupKey) {
+    ImmutableBitSet.Builder childKey = ImmutableBitSet.builder();
     RelMdUtil.setAggChildKeys(groupKey, rel, childKey);
-    return RelMetadataQuery.getPopulationSize(
-        rel.getInput(),
-        childKey);
+    return RelMetadataQuery.getPopulationSize(rel.getInput(), childKey.build());
   }
 
-  public Double getPopulationSize(Values rel, BitSet groupKey) {
+  public Double getPopulationSize(Values rel, ImmutableBitSet groupKey) {
     // assume half the rows are duplicates
     return rel.getRows() / 2;
   }
 
-  public Double getPopulationSize(Project rel, BitSet groupKey) {
-    BitSet baseCols = new BitSet();
-    BitSet projCols = new BitSet();
+  public Double getPopulationSize(Project rel, ImmutableBitSet groupKey) {
+    ImmutableBitSet.Builder baseCols = ImmutableBitSet.builder();
+    ImmutableBitSet.Builder projCols = ImmutableBitSet.builder();
     List<RexNode> projExprs = rel.getProjects();
     RelMdUtil.splitCols(projExprs, groupKey, baseCols, projCols);
 
     Double population =
         RelMetadataQuery.getPopulationSize(
             rel.getInput(),
-            baseCols);
+            baseCols.build());
     if (population == null) {
       return null;
     }
@@ -114,7 +109,7 @@ public class RelMdPopulationSize {
       return population;
     }
 
-    for (int bit : BitSets.toIter(projCols)) {
+    for (int bit : projCols.build()) {
       Double subRowCount =
           RelMdUtil.cardOfProjExpr(rel, projExprs.get(bit));
       if (subRowCount == null) {
@@ -132,7 +127,7 @@ public class RelMdPopulationSize {
   }
 
   // Catch-all rule when none of the others apply.
-  public Double getPopulationSize(RelNode rel, BitSet groupKey) {
+  public Double getPopulationSize(RelNode rel, ImmutableBitSet groupKey) {
     // if the keys are unique, return the row count; otherwise, we have
     // no further information on which to return any legitimate value
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/core/src/main/java/org/apache/calcite/rel/metadata/RelMdPredicates.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdPredicates.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdPredicates.java
index e15f074..39a4794 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdPredicates.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdPredicates.java
@@ -41,6 +41,7 @@ import org.apache.calcite.rex.RexVisitorImpl;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.util.BitSets;
 import org.apache.calcite.util.BuiltInMethod;
+import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.mapping.Mapping;
 import org.apache.calcite.util.mapping.MappingType;
 import org.apache.calcite.util.mapping.Mappings;
@@ -50,17 +51,16 @@ import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Iterators;
 import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
 
 import java.util.ArrayList;
 import java.util.BitSet;
-import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.SortedMap;
-import java.util.TreeMap;
 
 /**
  * Utility to infer Predicates that are applicable above a RelNode.
@@ -151,7 +151,7 @@ public class RelMdPredicates {
 
     List<RexNode> projectPullUpPredicates = new ArrayList<RexNode>();
 
-    BitSet columnsMapped = new BitSet(child.getRowType().getFieldCount());
+    ImmutableBitSet.Builder columnsMappedBuilder = ImmutableBitSet.builder();
     Mapping m = Mappings.create(MappingType.PARTIAL_FUNCTION,
         child.getRowType().getFieldCount(),
         project.getRowType().getFieldCount());
@@ -160,15 +160,16 @@ public class RelMdPredicates {
       if (o.e instanceof RexInputRef) {
         int sIdx = ((RexInputRef) o.e).getIndex();
         m.set(sIdx, o.i);
-        columnsMapped.set(sIdx);
+        columnsMappedBuilder.set(sIdx);
       }
     }
 
     // Go over childPullUpPredicates. If a predicate only contains columns in
     // 'columnsMapped' construct a new predicate based on mapping.
+    final ImmutableBitSet columnsMapped = columnsMappedBuilder.build();
     for (RexNode r : childInfo.pulledUpPredicates) {
-      BitSet rCols = RelOptUtil.InputFinder.bits(r);
-      if (BitSets.contains(columnsMapped, rCols)) {
+      ImmutableBitSet rCols = RelOptUtil.InputFinder.bits(r);
+      if (columnsMapped.contains(rCols)) {
         r = r.accept(new RexPermuteInputsShuttle(m, child));
         projectPullUpPredicates.add(r);
       }
@@ -235,18 +236,18 @@ public class RelMdPredicates {
 
     List<RexNode> aggPullUpPredicates = new ArrayList<RexNode>();
 
-    BitSet groupKeys = agg.getGroupSet();
+    ImmutableBitSet groupKeys = agg.getGroupSet();
     Mapping m = Mappings.create(MappingType.PARTIAL_FUNCTION,
         child.getRowType().getFieldCount(), agg.getRowType().getFieldCount());
 
     int i = 0;
-    for (int j : BitSets.toIter(groupKeys)) {
+    for (int j : groupKeys) {
       m.set(j, i++);
     }
 
     for (RexNode r : childInfo.pulledUpPredicates) {
-      BitSet rCols = RelOptUtil.InputFinder.bits(r);
-      if (BitSets.contains(groupKeys, rCols)) {
+      ImmutableBitSet rCols = RelOptUtil.InputFinder.bits(r);
+      if (groupKeys.contains(rCols)) {
         r = r.accept(new RexPermuteInputsShuttle(m, child));
         aggPullUpPredicates.add(r);
       }
@@ -316,11 +317,11 @@ public class RelMdPredicates {
     final int nSysFields;
     final int nFieldsLeft;
     final int nFieldsRight;
-    final BitSet leftFieldsBitSet;
-    final BitSet rightFieldsBitSet;
-    final BitSet allFieldsBitSet;
+    final ImmutableBitSet leftFieldsBitSet;
+    final ImmutableBitSet rightFieldsBitSet;
+    final ImmutableBitSet allFieldsBitSet;
     SortedMap<Integer, BitSet> equivalence;
-    final Map<String, BitSet> exprFields;
+    final Map<String, ImmutableBitSet> exprFields;
     final Set<String> allExprsDigests;
     final Set<String> equalityPredicates;
     final RexNode leftChildPredicates;
@@ -333,13 +334,14 @@ public class RelMdPredicates {
       nFieldsLeft = joinRel.getLeft().getRowType().getFieldList().size();
       nFieldsRight = joinRel.getRight().getRowType().getFieldList().size();
       nSysFields = joinRel.getSystemFieldList().size();
-      leftFieldsBitSet = BitSets.range(nSysFields, nSysFields + nFieldsLeft);
-      rightFieldsBitSet = BitSets.range(nSysFields + nFieldsLeft,
+      leftFieldsBitSet = ImmutableBitSet.range(nSysFields,
+          nSysFields + nFieldsLeft);
+      rightFieldsBitSet = ImmutableBitSet.range(nSysFields + nFieldsLeft,
           nSysFields + nFieldsLeft + nFieldsRight);
-      allFieldsBitSet = BitSets.range(0,
+      allFieldsBitSet = ImmutableBitSet.range(0,
           nSysFields + nFieldsLeft + nFieldsRight);
 
-      exprFields = new HashMap<String, BitSet>();
+      exprFields = Maps.newHashMap();
       allExprsDigests = new HashSet<String>();
 
       if (lPreds == null) {
@@ -370,7 +372,7 @@ public class RelMdPredicates {
         }
       }
 
-      equivalence = new TreeMap<Integer, BitSet>();
+      equivalence = Maps.newTreeMap();
       equalityPredicates = new HashSet<String>();
       for (int i = 0; i < nSysFields + nFieldsLeft + nFieldsRight; i++) {
         equivalence.put(i, BitSets.of(i));
@@ -444,10 +446,10 @@ public class RelMdPredicates {
       List<RexNode> rightInferredPredicates = new ArrayList<RexNode>();
 
       for (RexNode iP : inferredPredicates) {
-        BitSet iPBitSet = RelOptUtil.InputFinder.bits(iP);
-        if (BitSets.contains(leftFieldsBitSet, iPBitSet)) {
+        ImmutableBitSet iPBitSet = RelOptUtil.InputFinder.bits(iP);
+        if (leftFieldsBitSet.contains(iPBitSet)) {
           leftInferredPredicates.add(iP.accept(leftPermute));
-        } else if (BitSets.contains(rightFieldsBitSet, iPBitSet)) {
+        } else if (rightFieldsBitSet.contains(iPBitSet)) {
           rightInferredPredicates.add(iP.accept(rightPermute));
         }
       }
@@ -484,7 +486,7 @@ public class RelMdPredicates {
 
     private void infer(RexNode predicates, Set<String> allExprsDigests,
         List<RexNode> inferedPredicates, boolean includeEqualityInference,
-        BitSet inferringFields) {
+        ImmutableBitSet inferringFields) {
       for (RexNode r : RelOptUtil.conjunctions(predicates)) {
         if (!includeEqualityInference
             && equalityPredicates.contains(r.toString())) {
@@ -494,7 +496,7 @@ public class RelMdPredicates {
           RexNode tr = r.accept(
               new RexPermuteInputsShuttle(m, joinRel.getInput(0),
                   joinRel.getInput(1)));
-          if (BitSets.contains(inferringFields, RelOptUtil.InputFinder.bits(tr))
+          if (inferringFields.contains(RelOptUtil.InputFinder.bits(tr))
               && !allExprsDigests.contains(tr.toString())
               && !isAlwaysTrue(tr)) {
             inferedPredicates.add(tr);
@@ -507,7 +509,7 @@ public class RelMdPredicates {
     Iterable<Mapping> mappings(final RexNode predicate) {
       return new Iterable<Mapping>() {
         public Iterator<Mapping> iterator() {
-          BitSet fields = exprFields.get(predicate.toString());
+          ImmutableBitSet fields = exprFields.get(predicate.toString());
           if (fields.cardinality() == 0) {
             return Iterators.emptyIterator();
           }
@@ -596,7 +598,7 @@ public class RelMdPredicates {
       Mapping nextMapping;
       boolean firstCall;
 
-      ExprsItr(BitSet fields) {
+      ExprsItr(ImmutableBitSet fields) {
         nextMapping = null;
         columns = new int[fields.cardinality()];
         columnSets = new BitSet[fields.cardinality()];

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/core/src/main/java/org/apache/calcite/rel/metadata/RelMdRowCount.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdRowCount.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdRowCount.java
index 2549570..568f7ce 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdRowCount.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdRowCount.java
@@ -25,10 +25,9 @@ import org.apache.calcite.rel.core.Sort;
 import org.apache.calcite.rel.core.Union;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.util.BuiltInMethod;
+import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.NumberUtil;
 
-import java.util.BitSet;
-
 /**
  * RelMdRowCount supplies a default implementation of
  * {@link RelMetadataQuery#getRowCount} for the standard logical algebra.
@@ -83,10 +82,7 @@ public class RelMdRowCount {
   }
 
   public Double getRowCount(Aggregate rel) {
-    BitSet groupKey = new BitSet();
-    for (int i = 0; i < rel.getGroupCount(); i++) {
-      groupKey.set(i);
-    }
+    ImmutableBitSet groupKey = ImmutableBitSet.range(rel.getGroupCount());
 
     // rowcount is the cardinality of the group by columns
     Double distinctRowCount =

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/core/src/main/java/org/apache/calcite/rel/metadata/RelMdSelectivity.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdSelectivity.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdSelectivity.java
index 1706e13..3bffe26 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdSelectivity.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdSelectivity.java
@@ -28,8 +28,8 @@ import org.apache.calcite.rex.RexBuilder;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
-import org.apache.calcite.util.BitSets;
 import org.apache.calcite.util.BuiltInMethod;
+import org.apache.calcite.util.ImmutableBitSet;
 
 import java.util.ArrayList;
 import java.util.List;
@@ -157,7 +157,7 @@ public class RelMdSelectivity {
     List<RexNode> notPushable = new ArrayList<RexNode>();
     List<RexNode> pushable = new ArrayList<RexNode>();
     RelOptUtil.splitFilters(
-        BitSets.range(rel.getRowType().getFieldCount()),
+        ImmutableBitSet.range(rel.getRowType().getFieldCount()),
         predicate,
         pushable,
         notPushable);

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/core/src/main/java/org/apache/calcite/rel/metadata/RelMdUniqueKeys.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdUniqueKeys.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdUniqueKeys.java
index 55083d8..a84a94e 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdUniqueKeys.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdUniqueKeys.java
@@ -29,8 +29,10 @@ import org.apache.calcite.rex.RexInputRef;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.util.BitSets;
 import org.apache.calcite.util.BuiltInMethod;
+import org.apache.calcite.util.ImmutableBitSet;
+
+import com.google.common.collect.ImmutableSet;
 
-import java.util.BitSet;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
@@ -52,19 +54,20 @@ public class RelMdUniqueKeys {
 
   //~ Methods ----------------------------------------------------------------
 
-  public Set<BitSet> getUniqueKeys(Filter rel, boolean ignoreNulls) {
+  public Set<ImmutableBitSet> getUniqueKeys(Filter rel, boolean ignoreNulls) {
     return RelMetadataQuery.getUniqueKeys(rel.getInput(), ignoreNulls);
   }
 
-  public Set<BitSet> getUniqueKeys(Sort rel, boolean ignoreNulls) {
+  public Set<ImmutableBitSet> getUniqueKeys(Sort rel, boolean ignoreNulls) {
     return RelMetadataQuery.getUniqueKeys(rel.getInput(), ignoreNulls);
   }
 
-  public Set<BitSet> getUniqueKeys(Correlator rel, boolean ignoreNulls) {
+  public Set<ImmutableBitSet> getUniqueKeys(Correlator rel,
+      boolean ignoreNulls) {
     return RelMetadataQuery.getUniqueKeys(rel.getLeft(), ignoreNulls);
   }
 
-  public Set<BitSet> getUniqueKeys(Project rel, boolean ignoreNulls) {
+  public Set<ImmutableBitSet> getUniqueKeys(Project rel, boolean ignoreNulls) {
     // LogicalProject maps a set of rows to a different set;
     // Without knowledge of the mapping function(whether it
     // preserves uniqueness), it is only safe to derive uniqueness
@@ -76,7 +79,7 @@ public class RelMdUniqueKeys {
 
     List<RexNode> projExprs = rel.getProjects();
 
-    Set<BitSet> projUniqueKeySet = new HashSet<BitSet>();
+    Set<ImmutableBitSet> projUniqueKeySet = new HashSet<ImmutableBitSet>();
 
     // Build an input to output position map.
     for (int i = 0; i < projExprs.size(); i++) {
@@ -92,14 +95,14 @@ public class RelMdUniqueKeys {
       return projUniqueKeySet;
     }
 
-    Set<BitSet> childUniqueKeySet =
+    Set<ImmutableBitSet> childUniqueKeySet =
         RelMetadataQuery.getUniqueKeys(rel.getInput(), ignoreNulls);
 
     if (childUniqueKeySet != null) {
       // Now add to the projUniqueKeySet the child keys that are fully
       // projected.
-      for (BitSet colMask : childUniqueKeySet) {
-        BitSet tmpMask = new BitSet();
+      for (ImmutableBitSet colMask : childUniqueKeySet) {
+        ImmutableBitSet.Builder tmpMask = ImmutableBitSet.builder();
         boolean completeKeyProjected = true;
         for (int bit : BitSets.toIter(colMask)) {
           if (mapInToOutPos.containsKey(bit)) {
@@ -112,7 +115,7 @@ public class RelMdUniqueKeys {
           }
         }
         if (completeKeyProjected) {
-          projUniqueKeySet.add(tmpMask);
+          projUniqueKeySet.add(tmpMask.build());
         }
       }
     }
@@ -120,7 +123,7 @@ public class RelMdUniqueKeys {
     return projUniqueKeySet;
   }
 
-  public Set<BitSet> getUniqueKeys(Join rel, boolean ignoreNulls) {
+  public Set<ImmutableBitSet> getUniqueKeys(Join rel, boolean ignoreNulls) {
     final RelNode left = rel.getLeft();
     final RelNode right = rel.getRight();
 
@@ -133,31 +136,29 @@ public class RelMdUniqueKeys {
     // that is undesirable, use RelMetadataQuery.areColumnsUnique() as
     // an alternative way of getting unique key information.
 
-    Set<BitSet> retSet = new HashSet<BitSet>();
-    Set<BitSet> leftSet = RelMetadataQuery.getUniqueKeys(left, ignoreNulls);
-    Set<BitSet> rightSet = null;
+    Set<ImmutableBitSet> retSet = new HashSet<ImmutableBitSet>();
+    Set<ImmutableBitSet> leftSet =
+        RelMetadataQuery.getUniqueKeys(left, ignoreNulls);
+    Set<ImmutableBitSet> rightSet = null;
 
-    Set<BitSet> tmpRightSet =
+    Set<ImmutableBitSet> tmpRightSet =
         RelMetadataQuery.getUniqueKeys(right, ignoreNulls);
     int nFieldsOnLeft = left.getRowType().getFieldCount();
 
     if (tmpRightSet != null) {
-      rightSet = new HashSet<BitSet>();
-      for (BitSet colMask : tmpRightSet) {
-        BitSet tmpMask = new BitSet();
-        for (int bit : BitSets.toIter(colMask)) {
+      rightSet = new HashSet<ImmutableBitSet>();
+      for (ImmutableBitSet colMask : tmpRightSet) {
+        ImmutableBitSet.Builder tmpMask = ImmutableBitSet.builder();
+        for (int bit : colMask) {
           tmpMask.set(bit + nFieldsOnLeft);
         }
-        rightSet.add(tmpMask);
+        rightSet.add(tmpMask.build());
       }
 
       if (leftSet != null) {
-        for (BitSet colMaskRight : rightSet) {
-          for (BitSet colMaskLeft : leftSet) {
-            BitSet colMaskConcat = new BitSet();
-            colMaskConcat.or(colMaskLeft);
-            colMaskConcat.or(colMaskRight);
-            retSet.add(colMaskConcat);
+        for (ImmutableBitSet colMaskRight : rightSet) {
+          for (ImmutableBitSet colMaskLeft : leftSet) {
+            retSet.add(colMaskLeft.union(colMaskRight));
           }
         }
       }
@@ -196,28 +197,20 @@ public class RelMdUniqueKeys {
     return retSet;
   }
 
-  public Set<BitSet> getUniqueKeys(SemiJoin rel, boolean ignoreNulls) {
+  public Set<ImmutableBitSet> getUniqueKeys(SemiJoin rel, boolean ignoreNulls) {
     // only return the unique keys from the LHS since a semijoin only
     // returns the LHS
     return RelMetadataQuery.getUniqueKeys(rel.getLeft(), ignoreNulls);
   }
 
-  public Set<BitSet> getUniqueKeys(Aggregate rel, boolean ignoreNulls) {
-    Set<BitSet> retSet = new HashSet<BitSet>();
-
+  public Set<ImmutableBitSet> getUniqueKeys(Aggregate rel,
+      boolean ignoreNulls) {
     // group by keys form a unique key
-    if (rel.getGroupCount() > 0) {
-      BitSet groupKey = new BitSet();
-      for (int i = 0; i < rel.getGroupCount(); i++) {
-        groupKey.set(i);
-      }
-      retSet.add(groupKey);
-    }
-    return retSet;
+    return ImmutableSet.of(rel.getGroupSet());
   }
 
   // Catch-all rule when none of the others apply.
-  public Set<BitSet> getUniqueKeys(RelNode rel, boolean ignoreNulls) {
+  public Set<ImmutableBitSet> getUniqueKeys(RelNode rel, boolean ignoreNulls) {
     // no information available
     return null;
   }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/core/src/main/java/org/apache/calcite/rel/metadata/RelMdUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdUtil.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdUtil.java
index e3128fb..b9eb267 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdUtil.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdUtil.java
@@ -20,7 +20,6 @@ import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Aggregate;
 import org.apache.calcite.rel.core.AggregateCall;
-import org.apache.calcite.rel.core.JoinInfo;
 import org.apache.calcite.rel.core.JoinRelType;
 import org.apache.calcite.rel.core.Project;
 import org.apache.calcite.rel.core.SemiJoin;
@@ -36,15 +35,13 @@ import org.apache.calcite.sql.SqlFunctionCategory;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.type.OperandTypes;
 import org.apache.calcite.sql.type.ReturnTypes;
-import org.apache.calcite.util.BitSets;
-import org.apache.calcite.util.Bug;
+import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.NumberUtil;
 
 import com.google.common.collect.ImmutableList;
 
 import java.math.BigDecimal;
 import java.util.ArrayList;
-import java.util.BitSet;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
@@ -160,16 +157,18 @@ public class RelMdUtil {
       RelNode dimRel,
       List<Integer> factKeyList,
       List<Integer> dimKeyList) {
-    BitSet factKeys = new BitSet();
+    ImmutableBitSet.Builder factKeys = ImmutableBitSet.builder();
     for (int factCol : factKeyList) {
       factKeys.set(factCol);
     }
-    BitSet dimKeys = new BitSet();
+    ImmutableBitSet.Builder dimKeyBuilder = ImmutableBitSet.builder();
     for (int dimCol : dimKeyList) {
-      dimKeys.set(dimCol);
+      dimKeyBuilder.set(dimCol);
     }
+    final ImmutableBitSet dimKeys = dimKeyBuilder.build();
 
-    Double factPop = RelMetadataQuery.getPopulationSize(factRel, factKeys);
+    Double factPop =
+        RelMetadataQuery.getPopulationSize(factRel, factKeys.build());
     if (factPop == null) {
       // use the dimension population if the fact population is
       // unavailable; since we're filtering the fact table, that's
@@ -200,7 +199,7 @@ public class RelMdUtil {
       selectivity =
           Math.pow(
               0.1,
-              dimKeys.cardinality());
+              dimKeyBuilder.cardinality());
     } else if (selectivity > 1.0) {
       selectivity = 1.0;
     }
@@ -220,7 +219,7 @@ public class RelMdUtil {
    */
   public static boolean areColumnsDefinitelyUnique(
       RelNode rel,
-      BitSet colMask) {
+      ImmutableBitSet colMask) {
     Boolean b = RelMetadataQuery.areColumnsUnique(rel, colMask, false);
     return b != null && b;
   }
@@ -228,13 +227,11 @@ public class RelMdUtil {
   public static Boolean areColumnsUnique(
       RelNode rel,
       List<RexInputRef> columnRefs) {
-    BitSet colMask = new BitSet();
-
+    ImmutableBitSet.Builder colMask = ImmutableBitSet.builder();
     for (RexInputRef columnRef : columnRefs) {
       colMask.set(columnRef.getIndex());
     }
-
-    return RelMetadataQuery.areColumnsUnique(rel, colMask);
+    return RelMetadataQuery.areColumnsUnique(rel, colMask.build());
   }
 
   public static boolean areColumnsDefinitelyUnique(RelNode rel,
@@ -256,7 +253,7 @@ public class RelMdUtil {
    * if no metadata is available)
    */
   public static boolean areColumnsDefinitelyUniqueWhenNullsFiltered(RelNode rel,
-      BitSet colMask) {
+      ImmutableBitSet colMask) {
     Boolean b = RelMetadataQuery.areColumnsUnique(rel, colMask, true);
     if (b == null) {
       return false;
@@ -267,13 +264,13 @@ public class RelMdUtil {
   public static Boolean areColumnsUniqueWhenNullsFiltered(
       RelNode rel,
       List<RexInputRef> columnRefs) {
-    BitSet colMask = new BitSet();
+    ImmutableBitSet.Builder colMask = ImmutableBitSet.builder();
 
     for (RexInputRef columnRef : columnRefs) {
       colMask.set(columnRef.getIndex());
     }
 
-    return RelMetadataQuery.areColumnsUnique(rel, colMask, true);
+    return RelMetadataQuery.areColumnsUnique(rel, colMask.build(), true);
   }
 
   public static boolean areColumnsDefinitelyUniqueWhenNullsFiltered(
@@ -288,7 +285,7 @@ public class RelMdUtil {
 
   /**
    * Separates a bit-mask representing a join into masks representing the left
-   * and right inputs into the join
+   * and right inputs into the join.
    *
    * @param groupKey      original bit-mask
    * @param leftMask      left bit-mask to be set
@@ -296,11 +293,11 @@ public class RelMdUtil {
    * @param nFieldsOnLeft number of fields in the left input
    */
   public static void setLeftRightBitmaps(
-      BitSet groupKey,
-      BitSet leftMask,
-      BitSet rightMask,
+      ImmutableBitSet groupKey,
+      ImmutableBitSet.Builder leftMask,
+      ImmutableBitSet.Builder rightMask,
       int nFieldsOnLeft) {
-    for (int bit : BitSets.toIter(groupKey)) {
+    for (int bit : groupKey) {
       if (bit < nFieldsOnLeft) {
         leftMask.set(bit);
       } else {
@@ -434,35 +431,6 @@ public class RelMdUtil {
   }
 
   /**
-   * Locates the columns corresponding to equijoins within a joinrel.
-   *
-   * @param leftChild     left input into the join
-   * @param rightChild    right input into the join
-   * @param predicate     join predicate
-   * @param leftJoinCols  bitmap that will be set with the columns on the LHS
-   *                      of the join that participate in equijoins
-   * @param rightJoinCols bitmap that will be set with the columns on the RHS
-   *                      of the join that participate in equijoins
-   * @return remaining join filters that are not equijoins; may return a
-   * {@link RexLiteral} true, but never null
-   *
-   * @deprecated Will be removed after 0.9.1
-   */
-  public static RexNode findEquiJoinCols(
-      RelNode leftChild,
-      RelNode rightChild,
-      RexNode predicate,
-      BitSet leftJoinCols,
-      BitSet rightJoinCols) {
-    Bug.upgrade("remove after 0.9.1");
-    // locate the equijoin conditions
-    final JoinInfo joinInfo = JoinInfo.of(leftChild, rightChild, predicate);
-    BitSets.populate(leftJoinCols, joinInfo.leftKeys);
-    BitSets.populate(rightJoinCols, joinInfo.rightKeys);
-    return joinInfo.getRemaining(leftChild.getCluster().getRexBuilder());
-  }
-
-  /**
    * AND's two predicates together, either of which may be null, removing
    * redundant filters.
    *
@@ -527,18 +495,18 @@ public class RelMdUtil {
 
   /**
    * Takes a bitmap representing a set of input references and extracts the
-   * ones that reference the group by columns in an aggregate
+   * ones that reference the group by columns in an aggregate.
    *
    * @param groupKey the original bitmap
    * @param aggRel   the aggregate
    * @param childKey sets bits from groupKey corresponding to group by columns
    */
   public static void setAggChildKeys(
-      BitSet groupKey,
+      ImmutableBitSet groupKey,
       Aggregate aggRel,
-      BitSet childKey) {
+      ImmutableBitSet.Builder childKey) {
     List<AggregateCall> aggCalls = aggRel.getAggCallList();
-    for (int bit : BitSets.toIter(groupKey)) {
+    for (int bit : groupKey) {
       if (bit < aggRel.getGroupCount()) {
         // group by column
         childKey.set(bit);
@@ -556,7 +524,6 @@ public class RelMdUtil {
   /**
    * Forms two bitmaps by splitting the columns in a bitmap according to
    * whether or not the column references the child input or is an expression
-   *
    * @param projExprs Project expressions
    * @param groupKey  Bitmap whose columns will be split
    * @param baseCols  Bitmap representing columns from the child input
@@ -564,10 +531,10 @@ public class RelMdUtil {
    */
   public static void splitCols(
       List<RexNode> projExprs,
-      BitSet groupKey,
-      BitSet baseCols,
-      BitSet projCols) {
-    for (int bit : BitSets.toIter(groupKey)) {
+      ImmutableBitSet groupKey,
+      ImmutableBitSet.Builder baseCols,
+      ImmutableBitSet.Builder projCols) {
+    for (int bit : groupKey) {
       final RexNode e = projExprs.get(bit);
       if (e instanceof RexInputRef) {
         baseCols.set(((RexInputRef) e).getIndex());
@@ -598,9 +565,9 @@ public class RelMdUtil {
    */
   public static Double getJoinPopulationSize(
       RelNode joinRel,
-      BitSet groupKey) {
-    BitSet leftMask = new BitSet();
-    BitSet rightMask = new BitSet();
+      ImmutableBitSet groupKey) {
+    ImmutableBitSet.Builder leftMask = ImmutableBitSet.builder();
+    ImmutableBitSet.Builder rightMask = ImmutableBitSet.builder();
     RelNode left = joinRel.getInputs().get(0);
     RelNode right = joinRel.getInputs().get(1);
 
@@ -615,10 +582,10 @@ public class RelMdUtil {
         NumberUtil.multiply(
             RelMetadataQuery.getPopulationSize(
                 left,
-                leftMask),
+                leftMask.build()),
             RelMetadataQuery.getPopulationSize(
                 right,
-                rightMask));
+                rightMask.build()));
 
     return RelMdUtil.numDistinctVals(
         population,
@@ -640,12 +607,12 @@ public class RelMdUtil {
   public static Double getJoinDistinctRowCount(
       RelNode joinRel,
       JoinRelType joinType,
-      BitSet groupKey,
+      ImmutableBitSet groupKey,
       RexNode predicate,
       boolean useMaxNdv) {
     Double distRowCount;
-    BitSet leftMask = new BitSet();
-    BitSet rightMask = new BitSet();
+    ImmutableBitSet.Builder leftMask = ImmutableBitSet.builder();
+    ImmutableBitSet.Builder rightMask = ImmutableBitSet.builder();
     RelNode left = joinRel.getInputs().get(0);
     RelNode right = joinRel.getInputs().get(1);
 
@@ -685,22 +652,22 @@ public class RelMdUtil {
     if (useMaxNdv) {
       distRowCount = Math.max(RelMetadataQuery.getDistinctRowCount(
                 left,
-                leftMask,
+                leftMask.build(),
                 leftPred),
             RelMetadataQuery.getDistinctRowCount(
                 right,
-                rightMask,
+                rightMask.build(),
                 rightPred));
     } else {
       distRowCount =
         NumberUtil.multiply(
             RelMetadataQuery.getDistinctRowCount(
                 left,
-                leftMask,
+                leftMask.build(),
                 leftPred),
             RelMetadataQuery.getDistinctRowCount(
                 right,
-                rightMask,
+                rightMask.build(),
                 rightPred));
     }
 
@@ -723,8 +690,7 @@ public class RelMdUtil {
 
     public Double visitInputRef(RexInputRef var) {
       int index = var.getIndex();
-      BitSet col = new BitSet(index);
-      col.set(index);
+      ImmutableBitSet col = ImmutableBitSet.of(index);
       Double distinctRowCount =
           RelMetadataQuery.getDistinctRowCount(
               rel.getInput(),

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQuery.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQuery.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQuery.java
index 4d86f79..c537ef1 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQuery.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQuery.java
@@ -22,10 +22,10 @@ import org.apache.calcite.plan.RelOptTable;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.util.ImmutableBitSet;
 
 import com.google.common.collect.Iterables;
 
-import java.util.BitSet;
 import java.util.Set;
 
 /**
@@ -214,7 +214,7 @@ public abstract class RelMetadataQuery {
    * @return set of keys, or null if this information cannot be determined
    * (whereas empty set indicates definitely no keys at all)
    */
-  public static Set<BitSet> getUniqueKeys(RelNode rel) {
+  public static Set<ImmutableBitSet> getUniqueKeys(RelNode rel) {
     final BuiltInMetadata.UniqueKeys metadata =
         rel.metadata(BuiltInMetadata.UniqueKeys.class);
     return metadata.getUniqueKeys(false);
@@ -231,7 +231,8 @@ public abstract class RelMetadataQuery {
    * @return set of keys, or null if this information cannot be determined
    * (whereas empty set indicates definitely no keys at all)
    */
-  public static Set<BitSet> getUniqueKeys(RelNode rel, boolean ignoreNulls) {
+  public static Set<ImmutableBitSet> getUniqueKeys(RelNode rel,
+      boolean ignoreNulls) {
     final BuiltInMetadata.UniqueKeys metadata =
         rel.metadata(BuiltInMetadata.UniqueKeys.class);
     return metadata.getUniqueKeys(ignoreNulls);
@@ -239,7 +240,7 @@ public abstract class RelMetadataQuery {
 
   /**
    * Returns the
-   * {@link BuiltInMetadata.ColumnUniqueness#areColumnsUnique(BitSet, boolean)}
+   * {@link BuiltInMetadata.ColumnUniqueness#areColumnsUnique(org.apache.calcite.util.ImmutableBitSet, boolean)}
    * statistic.
    *
    * @param rel     the relational expression
@@ -248,7 +249,7 @@ public abstract class RelMetadataQuery {
    * @return true or false depending on whether the columns are unique, or
    * null if not enough information is available to make that determination
    */
-  public static Boolean areColumnsUnique(RelNode rel, BitSet columns) {
+  public static Boolean areColumnsUnique(RelNode rel, ImmutableBitSet columns) {
     final BuiltInMetadata.ColumnUniqueness metadata =
         rel.metadata(BuiltInMetadata.ColumnUniqueness.class);
     return metadata.areColumnsUnique(columns, false);
@@ -256,7 +257,7 @@ public abstract class RelMetadataQuery {
 
   /**
    * Returns the
-   * {@link BuiltInMetadata.ColumnUniqueness#areColumnsUnique(BitSet, boolean)}
+   * {@link BuiltInMetadata.ColumnUniqueness#areColumnsUnique(org.apache.calcite.util.ImmutableBitSet, boolean)}
    * statistic.
    *
    * @param rel         the relational expression
@@ -267,7 +268,7 @@ public abstract class RelMetadataQuery {
    * @return true or false depending on whether the columns are unique, or
    * null if not enough information is available to make that determination
    */
-  public static Boolean areColumnsUnique(RelNode rel, BitSet columns,
+  public static Boolean areColumnsUnique(RelNode rel, ImmutableBitSet columns,
       boolean ignoreNulls) {
     final BuiltInMetadata.ColumnUniqueness metadata =
         rel.metadata(BuiltInMetadata.ColumnUniqueness.class);
@@ -276,7 +277,7 @@ public abstract class RelMetadataQuery {
 
   /**
    * Returns the
-   * {@link BuiltInMetadata.PopulationSize#getPopulationSize(BitSet)}
+   * {@link BuiltInMetadata.PopulationSize#getPopulationSize(org.apache.calcite.util.ImmutableBitSet)}
    * statistic.
    *
    * @param rel      the relational expression
@@ -285,7 +286,8 @@ public abstract class RelMetadataQuery {
    * @return distinct row count for the given groupKey, or null if no reliable
    * estimate can be determined
    */
-  public static Double getPopulationSize(RelNode rel, BitSet groupKey) {
+  public static Double getPopulationSize(RelNode rel,
+      ImmutableBitSet groupKey) {
     final BuiltInMetadata.PopulationSize metadata =
         rel.metadata(BuiltInMetadata.PopulationSize.class);
     Double result = metadata.getPopulationSize(groupKey);
@@ -294,7 +296,7 @@ public abstract class RelMetadataQuery {
 
   /**
    * Returns the
-   * {@link BuiltInMetadata.DistinctRowCount#getDistinctRowCount(BitSet, RexNode)}
+   * {@link BuiltInMetadata.DistinctRowCount#getDistinctRowCount(org.apache.calcite.util.ImmutableBitSet, org.apache.calcite.rex.RexNode)}
    * statistic.
    *
    * @param rel       the relational expression
@@ -305,7 +307,7 @@ public abstract class RelMetadataQuery {
    */
   public static Double getDistinctRowCount(
       RelNode rel,
-      BitSet groupKey,
+      ImmutableBitSet groupKey,
       RexNode predicate) {
     final BuiltInMetadata.DistinctRowCount metadata =
         rel.metadata(BuiltInMetadata.DistinctRowCount.class);

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/core/src/main/java/org/apache/calcite/rel/rules/AggregateExpandDistinctAggregatesRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AggregateExpandDistinctAggregatesRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AggregateExpandDistinctAggregatesRule.java
index 915ae87..f513cad 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/AggregateExpandDistinctAggregatesRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/AggregateExpandDistinctAggregatesRule.java
@@ -31,7 +31,7 @@ import org.apache.calcite.rex.RexInputRef;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
-import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Util;
 
@@ -40,7 +40,6 @@ import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Lists;
 
 import java.util.ArrayList;
-import java.util.BitSet;
 import java.util.HashMap;
 import java.util.LinkedHashSet;
 import java.util.List;
@@ -124,12 +123,12 @@ public final class AggregateExpandDistinctAggregatesRule extends RelOptRule {
         aggregate.getRowType().getFieldList();
     final List<RexInputRef> refs = new ArrayList<RexInputRef>();
     final List<String> fieldNames = aggregate.getRowType().getFieldNames();
-    final BitSet groupSet = aggregate.getGroupSet();
+    final ImmutableBitSet groupSet = aggregate.getGroupSet();
     for (int i : Util.range(groupSet.cardinality())) {
       refs.add(RexInputRef.of(i, aggFields));
     }
 
-    // Aggregate the original relation, including any non-distinct aggs.
+    // Aggregate the original relation, including any non-distinct aggregates.
 
     List<AggregateCall> newAggCallList = new ArrayList<AggregateCall>();
     final int groupCount = groupSet.cardinality();
@@ -147,7 +146,7 @@ public final class AggregateExpandDistinctAggregatesRule extends RelOptRule {
       newAggCallList.add(aggCall);
     }
 
-    // In the case where there are no non-distinct aggs (regardless of
+    // In the case where there are no non-distinct aggregates (regardless of
     // whether there are group bys), there's no need to generate the
     // extra aggregate and join.
     RelNode rel;
@@ -207,7 +206,7 @@ public final class AggregateExpandDistinctAggregatesRule extends RelOptRule {
     return aggregate.copy(
         aggregate.getTraitSet(),
         distinct,
-        BitSets.range(aggregate.getGroupSet().cardinality()),
+        ImmutableBitSet.range(aggregate.getGroupSet().cardinality()),
         newAggCalls);
   }
 
@@ -346,7 +345,7 @@ public final class AggregateExpandDistinctAggregatesRule extends RelOptRule {
         aggregate.copy(
             aggregate.getTraitSet(),
             distinct,
-            BitSets.range(aggregate.getGroupSet().cardinality()),
+            ImmutableBitSet.range(aggregate.getGroupSet().cardinality()),
             aggCallList);
 
     // If there's no left child yet, no need to create the join
@@ -464,7 +463,7 @@ public final class AggregateExpandDistinctAggregatesRule extends RelOptRule {
     final RelNode child = aggregate.getInput();
     final List<RelDataTypeField> childFields =
         child.getRowType().getFieldList();
-    for (int i : BitSets.toIter(aggregate.getGroupSet())) {
+    for (int i : aggregate.getGroupSet()) {
       sourceOf.put(i, projects.size());
       projects.add(RexInputRef.of2(i, childFields));
     }
@@ -483,7 +482,7 @@ public final class AggregateExpandDistinctAggregatesRule extends RelOptRule {
     return aggregate.copy(
         aggregate.getTraitSet(),
         project,
-        BitSets.range(projects.size()),
+        ImmutableBitSet.range(projects.size()),
         ImmutableList.<AggregateCall>of());
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/core/src/main/java/org/apache/calcite/rel/rules/AggregateFilterTransposeRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AggregateFilterTransposeRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AggregateFilterTransposeRule.java
index 4b909cc..665191d 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/AggregateFilterTransposeRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/AggregateFilterTransposeRule.java
@@ -28,14 +28,13 @@ import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.sql.SqlAggFunction;
-import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.mapping.Mappings;
 
 import com.google.common.base.Function;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 
-import java.util.BitSet;
 import java.util.List;
 
 /**
@@ -68,10 +67,10 @@ public class AggregateFilterTransposeRule extends RelOptRule {
     final Filter filter = call.rel(1);
 
     // Do the columns used by the filter appear in the output of the aggregate?
-    final BitSet filterColumns =
+    final ImmutableBitSet filterColumns =
         RelOptUtil.InputFinder.bits(filter.getCondition());
-    final BitSet newGroupSet =
-        BitSets.union(aggregate.getGroupSet(), filterColumns);
+    final ImmutableBitSet newGroupSet =
+        aggregate.getGroupSet().union(filterColumns);
     final RelNode input = filter.getInput();
     final Boolean unique =
         RelMetadataQuery.areColumnsUnique(input, newGroupSet);
@@ -87,7 +86,7 @@ public class AggregateFilterTransposeRule extends RelOptRule {
     final Mappings.TargetMapping mapping = Mappings.target(
         new Function<Integer, Integer>() {
           public Integer apply(Integer a0) {
-            return BitSets.toList(newGroupSet).indexOf(a0);
+            return newGroupSet.indexOf(a0);
           }
         },
         input.getRowType().getFieldCount(),
@@ -96,16 +95,16 @@ public class AggregateFilterTransposeRule extends RelOptRule {
         RexUtil.apply(mapping, filter.getCondition());
     final Filter newFilter = filter.copy(filter.getTraitSet(),
         newAggregate, newCondition);
-    if (BitSets.contains(aggregate.getGroupSet(), filterColumns)) {
+    if (aggregate.getGroupSet().contains(filterColumns)) {
       // Everything needed by the filter is returned by the aggregate.
       assert newGroupSet.equals(aggregate.getGroupSet());
       call.transformTo(newFilter);
     } else {
       // The filter needs at least one extra column.
       // Now aggregate it away.
-      final BitSet topGroupSet = new BitSet();
-      for (int c : BitSets.toIter(aggregate.getGroupSet())) {
-        topGroupSet.set(BitSets.toList(newGroupSet).indexOf(c));
+      final ImmutableBitSet.Builder topGroupSet = ImmutableBitSet.builder();
+      for (int c : aggregate.getGroupSet()) {
+        topGroupSet.set(newGroupSet.indexOf(c));
       }
       final List<AggregateCall> topAggCallList = Lists.newArrayList();
       int i = newGroupSet.cardinality();
@@ -125,8 +124,8 @@ public class AggregateFilterTransposeRule extends RelOptRule {
                 ImmutableList.of(i++), aggregateCall.type, aggregateCall.name));
       }
       final Aggregate topAggregate =
-          aggregate.copy(aggregate.getTraitSet(), newFilter, topGroupSet,
-              topAggCallList);
+          aggregate.copy(aggregate.getTraitSet(), newFilter,
+              topGroupSet.build(), topAggCallList);
       call.transformTo(topAggregate);
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/core/src/main/java/org/apache/calcite/rel/rules/AggregateProjectMergeRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AggregateProjectMergeRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AggregateProjectMergeRule.java
index 8b1dafc..e321d83 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/AggregateProjectMergeRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/AggregateProjectMergeRule.java
@@ -26,12 +26,11 @@ import org.apache.calcite.rel.core.Project;
 import org.apache.calcite.rel.core.RelFactories;
 import org.apache.calcite.rex.RexInputRef;
 import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.ImmutableBitSet;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 
-import java.util.BitSet;
 import java.util.List;
 
 /**
@@ -68,7 +67,7 @@ public class AggregateProjectMergeRule extends RelOptRule {
   public static RelNode apply(Aggregate aggregate,
       Project project) {
     final List<Integer> newKeys = Lists.newArrayList();
-    for (int key : BitSets.toIter(aggregate.getGroupSet())) {
+    for (int key : aggregate.getGroupSet()) {
       final RexNode rex = project.getProjects().get(key);
       if (rex instanceof RexInputRef) {
         newKeys.add(((RexInputRef) rex).getIndex());
@@ -94,7 +93,7 @@ public class AggregateProjectMergeRule extends RelOptRule {
       aggCalls.add(aggregateCall.copy(newArgs.build()));
     }
 
-    final BitSet newGroupSet = BitSets.of(newKeys);
+    final ImmutableBitSet newGroupSet = ImmutableBitSet.of(newKeys);
     final Aggregate newAggregate =
         aggregate.copy(aggregate.getTraitSet(), project.getInput(), newGroupSet,
             aggCalls.build());
@@ -102,10 +101,10 @@ public class AggregateProjectMergeRule extends RelOptRule {
     // Add a project if the group set is not in the same order or
     // contains duplicates.
     RelNode rel = newAggregate;
-    if (!BitSets.toList(newGroupSet).equals(newKeys)) {
+    if (!newGroupSet.toList().equals(newKeys)) {
       final List<Integer> posList = Lists.newArrayList();
       for (int newKey : newKeys) {
-        posList.add(BitSets.toList(newGroupSet).indexOf(newKey));
+        posList.add(newGroupSet.indexOf(newKey));
       }
       for (int i = newAggregate.getGroupSet().cardinality();
            i < newAggregate.getRowType().getFieldCount(); i++) {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/core/src/main/java/org/apache/calcite/rel/rules/AggregateProjectPullUpConstantsRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AggregateProjectPullUpConstantsRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AggregateProjectPullUpConstantsRule.java
index cb99e86..d696407 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/AggregateProjectPullUpConstantsRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/AggregateProjectPullUpConstantsRule.java
@@ -29,7 +29,7 @@ import org.apache.calcite.rex.RexBuilder;
 import org.apache.calcite.rex.RexLocalRef;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexProgram;
-import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.IntList;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Permutation;
@@ -98,7 +98,7 @@ public class AggregateProjectPullUpConstantsRule extends RelOptRule {
     final RelDataType childRowType = child.getRowType();
     IntList constantList = new IntList();
     Map<Integer, RexNode> constants = new HashMap<Integer, RexNode>();
-    for (int i : BitSets.toIter(aggregate.getGroupSet())) {
+    for (int i : aggregate.getGroupSet()) {
       final RexLocalRef ref = program.getProjectList().get(i);
       if (program.isConstant(ref)) {
         constantList.add(i);
@@ -140,7 +140,7 @@ public class AggregateProjectPullUpConstantsRule extends RelOptRule {
           new LogicalAggregate(
               aggregate.getCluster(),
               child,
-              BitSets.range(newGroupCount),
+              ImmutableBitSet.range(newGroupCount),
               newAggCalls);
     } else {
       // Create the mapping from old field positions to new field
@@ -185,7 +185,7 @@ public class AggregateProjectPullUpConstantsRule extends RelOptRule {
           new LogicalAggregate(
               aggregate.getCluster(),
               project,
-              BitSets.range(newGroupCount),
+              ImmutableBitSet.range(newGroupCount),
               newAggCalls);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/core/src/main/java/org/apache/calcite/rel/rules/AggregateStarTableRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AggregateStarTableRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AggregateStarTableRule.java
index d511f79..019a01f 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/AggregateStarTableRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/AggregateStarTableRule.java
@@ -37,14 +37,13 @@ import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.schema.Table;
 import org.apache.calcite.schema.impl.StarTable;
 import org.apache.calcite.sql.SqlAggFunction;
-import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.mapping.AbstractSourceMapping;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 
-import java.util.BitSet;
 import java.util.List;
 
 /**
@@ -136,11 +135,11 @@ public class AggregateStarTableRule extends RelOptRule {
             + ", rolling up " + tileKey.dimensions + " to "
             + aggregate.getGroupSet());
       }
-      assert BitSets.contains(tileKey.dimensions, aggregate.getGroupSet());
+      assert tileKey.dimensions.contains(aggregate.getGroupSet());
       final List<AggregateCall> aggCalls = Lists.newArrayList();
-      BitSet groupSet = new BitSet();
-      for (int key : BitSets.toIter(aggregate.getGroupSet())) {
-        groupSet.set(BitSets.toList(tileKey.dimensions).indexOf(key));
+      ImmutableBitSet.Builder groupSet = ImmutableBitSet.builder();
+      for (int key : aggregate.getGroupSet()) {
+        groupSet.set(tileKey.dimensions.indexOf(key));
       }
       for (AggregateCall aggCall : aggregate.getAggCallList()) {
         final AggregateCall copy =
@@ -150,7 +149,8 @@ public class AggregateStarTableRule extends RelOptRule {
         }
         aggCalls.add(copy);
       }
-      rel = aggregate.copy(aggregate.getTraitSet(), rel, groupSet, aggCalls);
+      rel = aggregate.copy(aggregate.getTraitSet(), rel, groupSet.build(),
+          aggCalls);
     } else if (!tileKey.measures.equals(measures)) {
       System.out.println("Using materialization "
           + aggregateRelOptTable.getQualifiedName()
@@ -162,8 +162,8 @@ public class AggregateStarTableRule extends RelOptRule {
               aggregate.getRowType().getFieldCount()) {
             public int getSourceOpt(int source) {
               if (source < aggregate.getGroupCount()) {
-                int in = BitSets.toList(tileKey.dimensions).get(source);
-                return BitSets.toList(aggregate.getGroupSet()).indexOf(in);
+                int in = tileKey.dimensions.nth(source);
+                return aggregate.getGroupSet().indexOf(in);
               }
               Lattice.Measure measure =
                   measures.get(source - aggregate.getGroupCount());
@@ -184,8 +184,7 @@ public class AggregateStarTableRule extends RelOptRule {
     if (aggregateCall.isDistinct()) {
       return null;
     }
-    final SqlAggFunction aggregation =
-        (SqlAggFunction) aggregateCall.getAggregation();
+    final SqlAggFunction aggregation = aggregateCall.getAggregation();
     final Pair<SqlAggFunction, List<Integer>> seek =
         Pair.of(aggregation, aggregateCall.getArgList());
     final int offset = tileKey.dimensions.cardinality();
@@ -209,13 +208,13 @@ public class AggregateStarTableRule extends RelOptRule {
     {
       List<Integer> newArgs = Lists.newArrayList();
       for (Integer arg : aggregateCall.getArgList()) {
-        int z = BitSets.toList(tileKey.dimensions).indexOf(arg);
+        int z = tileKey.dimensions.indexOf(arg);
         if (z < 0) {
           break tryGroup;
         }
         newArgs.add(z);
       }
-      return AggregateCall.create((SqlAggFunction) aggregation, false, newArgs,
+      return AggregateCall.create(aggregation, false, newArgs,
           groupCount, input, null, aggregateCall.name);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/core/src/main/java/org/apache/calcite/rel/rules/FilterAggregateTransposeRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/FilterAggregateTransposeRule.java b/core/src/main/java/org/apache/calcite/rel/rules/FilterAggregateTransposeRule.java
index c3a7e25..8af01e3 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/FilterAggregateTransposeRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/FilterAggregateTransposeRule.java
@@ -26,12 +26,11 @@ import org.apache.calcite.rel.core.RelFactories;
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rex.RexBuilder;
 import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.ImmutableBitSet;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 
-import java.util.BitSet;
 import java.util.List;
 
 /**
@@ -80,7 +79,7 @@ public class FilterAggregateTransposeRule extends RelOptRule {
 
     final List<RexNode> conditions =
         RelOptUtil.conjunctions(filterRel.getCondition());
-    final BitSet groupKeys = aggRel.getGroupSet();
+    final ImmutableBitSet groupKeys = aggRel.getGroupSet();
     final RexBuilder rexBuilder = filterRel.getCluster().getRexBuilder();
     final List<RelDataTypeField> origFields =
         aggRel.getRowType().getFieldList();
@@ -89,8 +88,8 @@ public class FilterAggregateTransposeRule extends RelOptRule {
     final List<RexNode> remainingConditions = Lists.newArrayList();
 
     for (RexNode condition : conditions) {
-      BitSet rCols = RelOptUtil.InputFinder.bits(condition);
-      if (BitSets.contains(groupKeys, rCols)) {
+      ImmutableBitSet rCols = RelOptUtil.InputFinder.bits(condition);
+      if (groupKeys.contains(rCols)) {
         pushedConditions.add(
             condition.accept(
                 new RelOptUtil.RexInputConverter(rexBuilder, origFields,

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/core/src/main/java/org/apache/calcite/rel/rules/JoinAssociateRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/JoinAssociateRule.java b/core/src/main/java/org/apache/calcite/rel/rules/JoinAssociateRule.java
index 6a38a9a..455176f 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/JoinAssociateRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/JoinAssociateRule.java
@@ -27,12 +27,11 @@ import org.apache.calcite.rex.RexBuilder;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexPermuteInputsShuttle;
 import org.apache.calcite.rex.RexUtil;
-import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.mapping.Mappings;
 
 import com.google.common.collect.Lists;
 
-import java.util.BitSet;
 import java.util.List;
 
 /**
@@ -90,8 +89,9 @@ public class JoinAssociateRule extends RelOptRule {
     final int aCount = relA.getRowType().getFieldCount();
     final int bCount = relB.getRowType().getFieldCount();
     final int cCount = relC.getRowType().getFieldCount();
-    final BitSet aBitSet = BitSets.range(0, aCount);
-    final BitSet bBitSet = BitSets.range(aCount, aCount + bCount);
+    final ImmutableBitSet aBitSet = ImmutableBitSet.range(0, aCount);
+    final ImmutableBitSet bBitSet =
+        ImmutableBitSet.range(aCount, aCount + bCount);
 
     if (!topJoin.getSystemFieldList().isEmpty()) {
       // FIXME Enable this rule for joins with system fields

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/core/src/main/java/org/apache/calcite/rel/rules/JoinPushThroughJoinRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/JoinPushThroughJoinRule.java b/core/src/main/java/org/apache/calcite/rel/rules/JoinPushThroughJoinRule.java
index 8e024aa..6a292f6 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/JoinPushThroughJoinRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/JoinPushThroughJoinRule.java
@@ -30,11 +30,10 @@ import org.apache.calcite.rex.RexBuilder;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexPermuteInputsShuttle;
 import org.apache.calcite.rex.RexUtil;
-import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.mapping.Mappings;
 
 import java.util.ArrayList;
-import java.util.BitSet;
 import java.util.List;
 
 /**
@@ -118,7 +117,8 @@ public class JoinPushThroughJoinRule extends RelOptRule {
     final int aCount = relA.getRowType().getFieldCount();
     final int bCount = relB.getRowType().getFieldCount();
     final int cCount = relC.getRowType().getFieldCount();
-    final BitSet bBitSet = BitSets.range(aCount, aCount + bCount);
+    final ImmutableBitSet bBitSet =
+        ImmutableBitSet.range(aCount, aCount + bCount);
 
     // becomes
     //
@@ -225,7 +225,7 @@ public class JoinPushThroughJoinRule extends RelOptRule {
     final int aCount = relA.getRowType().getFieldCount();
     final int bCount = relB.getRowType().getFieldCount();
     final int cCount = relC.getRowType().getFieldCount();
-    final BitSet aBitSet = BitSets.range(aCount);
+    final ImmutableBitSet aBitSet = ImmutableBitSet.range(aCount);
 
     // becomes
     //
@@ -317,11 +317,11 @@ public class JoinPushThroughJoinRule extends RelOptRule {
    */
   static void split(
       RexNode condition,
-      BitSet bitSet,
+      ImmutableBitSet bitSet,
       List<RexNode> intersecting,
       List<RexNode> nonIntersecting) {
     for (RexNode node : RelOptUtil.conjunctions(condition)) {
-      BitSet inputBitSet = RelOptUtil.InputFinder.bits(node);
+      ImmutableBitSet inputBitSet = RelOptUtil.InputFinder.bits(node);
       if (bitSet.intersects(inputBitSet)) {
         intersecting.add(node);
       } else {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/core/src/main/java/org/apache/calcite/rel/rules/JoinToMultiJoinRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/JoinToMultiJoinRule.java b/core/src/main/java/org/apache/calcite/rel/rules/JoinToMultiJoinRule.java
index 082575a..fe7fa8c 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/JoinToMultiJoinRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/JoinToMultiJoinRule.java
@@ -29,6 +29,7 @@ import org.apache.calcite.rex.RexInputRef;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.ImmutableIntList;
 import org.apache.calcite.util.Pair;
 
@@ -36,7 +37,6 @@ import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 
-import java.util.BitSet;
 import java.util.List;
 import java.util.Map;
 
@@ -125,7 +125,7 @@ public class JoinToMultiJoinRule extends RelOptRule {
 
     // combine the children MultiJoin inputs into an array of inputs
     // for the new MultiJoin
-    final List<BitSet> projFieldsList = Lists.newArrayList();
+    final List<ImmutableBitSet> projFieldsList = Lists.newArrayList();
     final List<int[]> joinFieldRefCountsList = Lists.newArrayList();
     final List<RelNode> newInputs =
         combineInputs(
@@ -195,7 +195,7 @@ public class JoinToMultiJoinRule extends RelOptRule {
       Join join,
       RelNode left,
       RelNode right,
-      List<BitSet> projFieldsList,
+      List<ImmutableBitSet> projFieldsList,
       List<int[]> joinFieldRefCountsList) {
     final List<RelNode> newInputs = Lists.newArrayList();
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/core/src/main/java/org/apache/calcite/rel/rules/LoptMultiJoin.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/LoptMultiJoin.java b/core/src/main/java/org/apache/calcite/rel/rules/LoptMultiJoin.java
index 071dbc1..e75cc7c 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/LoptMultiJoin.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/LoptMultiJoin.java
@@ -29,10 +29,12 @@ import org.apache.calcite.rex.RexCall;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.IntList;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
 
 import java.util.ArrayList;
 import java.util.BitSet;
@@ -94,14 +96,14 @@ public class LoptMultiJoin {
    * bitmap contains the non-null generating factors that the null generating
    * factor is dependent upon
    */
-  private final BitSet [] outerJoinFactors;
+  private final ImmutableBitSet [] outerJoinFactors;
 
   /**
    * Bitmap corresponding to the fields projected from each join factor, after
    * row scan processing has completed. This excludes fields referenced in
    * join conditions, unless the field appears in the final projection list.
    */
-  private List<BitSet> projFields;
+  private List<ImmutableBitSet> projFields;
 
   /**
    * Map containing reference counts of the fields referenced in join
@@ -116,13 +118,13 @@ public class LoptMultiJoin {
    * For each join filter, associates a bitmap indicating all factors
    * referenced by the filter
    */
-  private Map<RexNode, BitSet> factorsRefByJoinFilter;
+  private Map<RexNode, ImmutableBitSet> factorsRefByJoinFilter;
 
   /**
    * For each join filter, associates a bitmap indicating all fields
    * referenced by the filter
    */
-  private Map<RexNode, BitSet> fieldsRefByJoinFilter;
+  private Map<RexNode, ImmutableBitSet> fieldsRefByJoinFilter;
 
   /**
    * Starting RexInputRef index corresponding to each join factor
@@ -138,7 +140,7 @@ public class LoptMultiJoin {
    * Bitmap indicating which factors each factor references in join filters
    * that correspond to comparisons
    */
-  BitSet [] factorsRefByFactor;
+  ImmutableBitSet [] factorsRefByFactor;
 
   /**
    * Weights of each factor combination
@@ -212,15 +214,15 @@ public class LoptMultiJoin {
     // upon.
     joinTypes = ImmutableList.copyOf(multiJoin.getJoinTypes());
     List<RexNode> outerJoinConds = this.multiJoin.getOuterJoinConditions();
-    outerJoinFactors = new BitSet[nJoinFactors];
+    outerJoinFactors = new ImmutableBitSet[nJoinFactors];
     for (int i = 0; i < nJoinFactors; i++) {
       if (outerJoinConds.get(i) != null) {
         // set a bitmap containing the factors referenced in the
         // ON condition of the outer join; mask off the factor
         // corresponding to the factor itself
-        BitSet dependentFactors =
+        ImmutableBitSet dependentFactors =
             getJoinFilterFactorBitmap(outerJoinConds.get(i), false);
-        dependentFactors.clear(i);
+        dependentFactors = dependentFactors.clear(i);
         outerJoinFactors[i] = dependentFactors;
       }
     }
@@ -291,7 +293,7 @@ public class LoptMultiJoin {
    * @return bitmap corresponding to the factors referenced within the
    * specified join filter
    */
-  public BitSet getFactorsRefByJoinFilter(RexNode joinFilter) {
+  public ImmutableBitSet getFactorsRefByJoinFilter(RexNode joinFilter) {
     return factorsRefByJoinFilter.get(joinFilter);
   }
 
@@ -307,7 +309,7 @@ public class LoptMultiJoin {
    *
    * @return bitmap corresponding to the fields referenced by a join filter
    */
-  public BitSet getFieldsRefByJoinFilter(RexNode joinFilter) {
+  public ImmutableBitSet getFieldsRefByJoinFilter(RexNode joinFilter) {
     return fieldsRefByJoinFilter.get(joinFilter);
   }
 
@@ -324,7 +326,7 @@ public class LoptMultiJoin {
    * @return bitmap corresponding to the factors referenced by the specified
    * factor in the various join filters that correspond to comparisons
    */
-  public BitSet getFactorsRefByFactor(int factIdx) {
+  public ImmutableBitSet getFactorsRefByFactor(int factIdx) {
     return factorsRefByFactor[factIdx];
   }
 
@@ -354,7 +356,7 @@ public class LoptMultiJoin {
    * dependent upon, if the factor is null generating in a left or right outer
    * join; otherwise null is returned
    */
-  public BitSet getOuterJoinFactors(int factIdx) {
+  public ImmutableBitSet getOuterJoinFactors(int factIdx) {
     return outerJoinFactors[factIdx];
   }
 
@@ -373,7 +375,7 @@ public class LoptMultiJoin {
    *
    * @return bitmap containing the fields that are projected from a factor
    */
-  public BitSet getProjFields(int factIdx) {
+  public ImmutableBitSet getProjFields(int factIdx) {
     return projFields.get(factIdx);
   }
 
@@ -438,10 +440,10 @@ public class LoptMultiJoin {
    *
    * @return the bitmap containing the factor references
    */
-  BitSet getJoinFilterFactorBitmap(
+  ImmutableBitSet getJoinFilterFactorBitmap(
       RexNode joinFilter,
       boolean setFields) {
-    BitSet fieldRefBitmap = fieldBitmap(joinFilter);
+    ImmutableBitSet fieldRefBitmap = fieldBitmap(joinFilter);
     if (setFields) {
       fieldsRefByJoinFilter.put(joinFilter, fieldRefBitmap);
     }
@@ -449,10 +451,10 @@ public class LoptMultiJoin {
     return factorBitmap(fieldRefBitmap);
   }
 
-  private BitSet fieldBitmap(RexNode joinFilter) {
-    BitSet fieldRefBitmap = new BitSet(nTotalFields);
-    joinFilter.accept(new RelOptUtil.InputFinder(fieldRefBitmap));
-    return fieldRefBitmap;
+  private ImmutableBitSet fieldBitmap(RexNode joinFilter) {
+    final RelOptUtil.InputFinder inputFinder = new RelOptUtil.InputFinder();
+    joinFilter.accept(inputFinder);
+    return inputFinder.inputBitSet.build();
   }
 
   /**
@@ -460,8 +462,8 @@ public class LoptMultiJoin {
    * references
    */
   private void setJoinFilterRefs() {
-    fieldsRefByJoinFilter = new HashMap<RexNode, BitSet>();
-    factorsRefByJoinFilter = new HashMap<RexNode, BitSet>();
+    fieldsRefByJoinFilter = Maps.newHashMap();
+    factorsRefByJoinFilter = Maps.newHashMap();
     ListIterator<RexNode> filterIter = allJoinFilters.listIterator();
     while (filterIter.hasNext()) {
       RexNode joinFilter = filterIter.next();
@@ -471,7 +473,7 @@ public class LoptMultiJoin {
       if (joinFilter.isAlwaysTrue()) {
         filterIter.remove();
       }
-      BitSet factorRefBitmap =
+      ImmutableBitSet factorRefBitmap =
           getJoinFilterFactorBitmap(joinFilter, true);
       factorsRefByJoinFilter.put(joinFilter, factorRefBitmap);
     }
@@ -485,13 +487,13 @@ public class LoptMultiJoin {
    * @return bitmap representing factors referenced that will
    * be set by this method
    */
-  private BitSet factorBitmap(BitSet fieldRefBitmap) {
-    BitSet factorRefBitmap = new BitSet(nJoinFactors);
-    for (int field : BitSets.toIter(fieldRefBitmap)) {
+  private ImmutableBitSet factorBitmap(ImmutableBitSet fieldRefBitmap) {
+    ImmutableBitSet.Builder factorRefBitmap = ImmutableBitSet.builder();
+    for (int field : fieldRefBitmap) {
       int factor = findRef(field);
       factorRefBitmap.set(factor);
     }
-    return factorRefBitmap;
+    return factorRefBitmap.build();
   }
 
   /**
@@ -519,13 +521,13 @@ public class LoptMultiJoin {
    */
   public void setFactorWeights() {
     factorWeights = new int[nJoinFactors][nJoinFactors];
-    factorsRefByFactor = new BitSet[nJoinFactors];
+    factorsRefByFactor = new ImmutableBitSet[nJoinFactors];
     for (int i = 0; i < nJoinFactors; i++) {
-      factorsRefByFactor[i] = new BitSet(nJoinFactors);
+      factorsRefByFactor[i] = ImmutableBitSet.of();
     }
 
     for (RexNode joinFilter : allJoinFilters) {
-      BitSet factorRefs = factorsRefByJoinFilter.get(joinFilter);
+      ImmutableBitSet factorRefs = factorsRefByJoinFilter.get(joinFilter);
 
       // don't give weights to non-comparison expressions
       if (!(joinFilter instanceof RexCall)) {
@@ -538,9 +540,12 @@ public class LoptMultiJoin {
       // OR the factors referenced in this join filter into the
       // bitmaps corresponding to each of the factors; however,
       // exclude the bit corresponding to the factor itself
-      for (int factor : BitSets.toIter(factorRefs)) {
-        factorsRefByFactor[factor].or(factorRefs);
-        factorsRefByFactor[factor].clear(factor);
+      for (int factor : factorRefs) {
+        factorsRefByFactor[factor] =
+            ImmutableBitSet.builder(factorsRefByFactor[factor])
+                .addAll(factorRefs)
+                .clear(factor)
+                .build();
       }
 
       if (factorRefs.cardinality() == 2) {
@@ -548,8 +553,8 @@ public class LoptMultiJoin {
         int rightFactor = factorRefs.nextSetBit(leftFactor + 1);
 
         final RexCall call = (RexCall) joinFilter;
-        BitSet leftFields = fieldBitmap(call.getOperands().get(0));
-        BitSet leftBitmap = factorBitmap(leftFields);
+        ImmutableBitSet leftFields = fieldBitmap(call.getOperands().get(0));
+        ImmutableBitSet leftBitmap = factorBitmap(leftFields);
 
         // filter contains only two factor references, one on each
         // side of the operator
@@ -571,7 +576,7 @@ public class LoptMultiJoin {
       } else {
         // multiple factor references -- set a weight for each
         // combination of factors referenced within the filter
-        final IntList list = BitSets.toList(factorRefs);
+        final IntList list = factorRefs.toList();
         for (int outer : list) {
           for (int inner : list) {
             if (outer != inner) {
@@ -618,7 +623,8 @@ public class LoptMultiJoin {
    * @param joinTree join tree to be examined
    * @param childFactors bitmap to be set
    */
-  public void getChildFactors(LoptJoinTree joinTree, BitSet childFactors) {
+  public void getChildFactors(LoptJoinTree joinTree,
+      ImmutableBitSet.Builder childFactors) {
     for (int child : joinTree.getTreeOrder()) {
       childFactors.set(child);
     }
@@ -791,18 +797,18 @@ public class LoptMultiJoin {
   }
 
   public Edge createEdge(RexNode condition) {
-    BitSet fieldRefBitmap = fieldBitmap(condition);
-    BitSet factorRefBitmap = factorBitmap(fieldRefBitmap);
+    ImmutableBitSet fieldRefBitmap = fieldBitmap(condition);
+    ImmutableBitSet factorRefBitmap = factorBitmap(fieldRefBitmap);
     return new Edge(condition, factorRefBitmap, fieldRefBitmap);
   }
 
   /** Information about a join-condition. */
   static class Edge {
-    final BitSet factors;
-    final BitSet columns;
+    final ImmutableBitSet factors;
+    final ImmutableBitSet columns;
     final RexNode condition;
 
-    Edge(RexNode condition, BitSet factors, BitSet columns) {
+    Edge(RexNode condition, ImmutableBitSet factors, ImmutableBitSet columns) {
       this.condition = condition;
       this.factors = factors;
       this.columns = columns;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/core/src/main/java/org/apache/calcite/rel/rules/LoptOptimizeJoinRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/LoptOptimizeJoinRule.java b/core/src/main/java/org/apache/calcite/rel/rules/LoptOptimizeJoinRule.java
index 2f5643b..7a365f3 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/LoptOptimizeJoinRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/LoptOptimizeJoinRule.java
@@ -41,6 +41,7 @@ import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.ImmutableIntList;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.mapping.IntPair;
@@ -150,7 +151,7 @@ public class LoptOptimizeJoinRule extends RelOptRule {
     outerForLoop:
       for (int factIdx : removalCandidates) {
         // reject the factor if it is referenced in the projection list
-        BitSet projFields = multiJoin.getProjFields(factIdx);
+        ImmutableBitSet projFields = multiJoin.getProjFields(factIdx);
         if ((projFields == null) || (projFields.cardinality() > 0)) {
           continue;
         }
@@ -163,9 +164,10 @@ public class LoptOptimizeJoinRule extends RelOptRule {
         List<RexNode> ojFilters = new ArrayList<RexNode>();
         RelOptUtil.decomposeConjunction(outerJoinCond, ojFilters);
         int numFields = multiJoin.getNumFieldsInJoinFactor(factIdx);
-        BitSet joinKeys = new BitSet(numFields);
-        BitSet otherJoinKeys =
-            new BitSet(multiJoin.getNumTotalFields());
+        final ImmutableBitSet.Builder joinKeyBuilder =
+            ImmutableBitSet.builder();
+        final ImmutableBitSet.Builder otherJoinKeyBuilder =
+            ImmutableBitSet.builder();
         int firstFieldNum = multiJoin.getJoinStart(factIdx);
         int lastFieldNum = firstFieldNum + numFields;
         for (RexNode filter : ojFilters) {
@@ -185,8 +187,8 @@ public class LoptOptimizeJoinRule extends RelOptRule {
           int rightRef =
               ((RexInputRef) filterCall.getOperands().get(1)).getIndex();
           setJoinKey(
-              joinKeys,
-              otherJoinKeys,
+              joinKeyBuilder,
+              otherJoinKeyBuilder,
               leftRef,
               rightRef,
               firstFieldNum,
@@ -194,12 +196,13 @@ public class LoptOptimizeJoinRule extends RelOptRule {
               true);
         }
 
-        if (joinKeys.cardinality() == 0) {
+        if (joinKeyBuilder.cardinality() == 0) {
           continue;
         }
 
         // make sure the only join fields referenced are the ones in
         // the current outer join
+        final ImmutableBitSet joinKeys = joinKeyBuilder.build();
         int [] joinFieldRefCounts =
             multiJoin.getJoinFieldRefCounts(factIdx);
         for (int i = 0; i < joinFieldRefCounts.length; i++) {
@@ -223,7 +226,8 @@ public class LoptOptimizeJoinRule extends RelOptRule {
           // the join keys from the other factors that join with
           // this one.  Later, in the outermost loop, we'll have
           // the opportunity to retry removing those factors.
-          for (int otherKey : BitSets.toIter(otherJoinKeys)) {
+          final ImmutableBitSet otherJoinKeys = otherJoinKeyBuilder.build();
+          for (int otherKey : otherJoinKeys) {
             int otherFactor = multiJoin.findRef(otherKey);
             if (multiJoin.isNullGenerating(otherFactor)) {
               retryCandidates.add(otherFactor);
@@ -256,8 +260,8 @@ public class LoptOptimizeJoinRule extends RelOptRule {
    * one
    */
   private void setJoinKey(
-      BitSet joinKeys,
-      BitSet otherJoinKeys,
+      ImmutableBitSet.Builder joinKeys,
+      ImmutableBitSet.Builder otherJoinKeys,
       int ref1,
       int ref2,
       int firstFieldNum,
@@ -325,7 +329,7 @@ public class LoptOptimizeJoinRule extends RelOptRule {
       int factor2 = selfJoinPairs.get(factor1);
       List<RexNode> selfJoinFilters = new ArrayList<RexNode>();
       for (RexNode filter : multiJoin.getJoinFilters()) {
-        BitSet joinFactors =
+        ImmutableBitSet joinFactors =
             multiJoin.getFactorsRefByJoinFilter(filter);
         if ((joinFactors.cardinality() == 2)
             && joinFactors.get(factor1)
@@ -571,12 +575,15 @@ public class LoptOptimizeJoinRule extends RelOptRule {
       LoptJoinTree joinTree,
       List<RexNode> filters,
       int factor) {
-    BitSet childFactors = BitSets.of(joinTree.getTreeOrder());
-    childFactors.set(factor);
+    final ImmutableBitSet childFactors =
+        ImmutableBitSet.builder()
+            .addAll(joinTree.getTreeOrder())
+            .set(factor)
+            .build();
 
     int factorStart = multiJoin.getJoinStart(factor);
     int nFields = multiJoin.getNumFieldsInJoinFactor(factor);
-    BitSet joinKeys = new BitSet(nFields);
+    final ImmutableBitSet.Builder joinKeys = ImmutableBitSet.builder();
 
     // first loop through the inner join filters, picking out the ones
     // that reference only the factors in either the join tree or the
@@ -607,7 +614,7 @@ public class LoptOptimizeJoinRule extends RelOptRule {
     } else {
       return RelMetadataQuery.getDistinctRowCount(
           semiJoinOpt.getChosenSemiJoin(factor),
-          joinKeys,
+          joinKeys.build(),
           null);
     }
   }
@@ -630,12 +637,12 @@ public class LoptOptimizeJoinRule extends RelOptRule {
   private void setFactorJoinKeys(
       LoptMultiJoin multiJoin,
       List<RexNode> filters,
-      BitSet joinFactors,
+      ImmutableBitSet joinFactors,
       int factorStart,
       int nFields,
-      BitSet joinKeys) {
+      ImmutableBitSet.Builder joinKeys) {
     for (RexNode joinFilter : filters) {
-      BitSet filterFactors =
+      ImmutableBitSet filterFactors =
           multiJoin.getFactorsRefByJoinFilter(joinFilter);
 
       // if all factors in the join filter are in the bitmap containing
@@ -643,8 +650,8 @@ public class LoptOptimizeJoinRule extends RelOptRule {
       // fields corresponding to the specified factor to the join key
       // bitmap; in doing so, adjust the join keys so they start at
       // offset 0
-      if (BitSets.contains(joinFactors, filterFactors)) {
-        BitSet joinFields =
+      if (joinFactors.contains(filterFactors)) {
+        ImmutableBitSet joinFields =
             multiJoin.getFieldsRefByJoinFilter(joinFilter);
         for (int field = joinFields.nextSetBit(factorStart);
              (field >= 0)
@@ -711,9 +718,9 @@ public class LoptOptimizeJoinRule extends RelOptRule {
       // this factor joins with that have already been added to
       // the tree
       BitSet factorsNeeded =
-          (BitSet) multiJoin.getFactorsRefByFactor(nextFactor).clone();
+          multiJoin.getFactorsRefByFactor(nextFactor).toBitSet();
       if (multiJoin.isNullGenerating(nextFactor)) {
-        factorsNeeded.or(multiJoin.getOuterJoinFactors(nextFactor));
+        factorsNeeded.or(multiJoin.getOuterJoinFactors(nextFactor).toBitSet());
       }
       factorsNeeded.and(factorsAdded);
       joinTree =
@@ -1243,24 +1250,27 @@ public class LoptOptimizeJoinRule extends RelOptRule {
     // ones that reference only the factors in the new join tree
     final RexBuilder rexBuilder =
         multiJoin.getMultiJoinRel().getCluster().getRexBuilder();
-    final BitSet childFactors = BitSets.of(rightTree.getTreeOrder());
+    final ImmutableBitSet.Builder childFactorBuilder =
+        ImmutableBitSet.builder();
+    childFactorBuilder.addAll(rightTree.getTreeOrder());
     if (leftIdx >= 0) {
-      childFactors.set(leftIdx);
+      childFactorBuilder.set(leftIdx);
     } else {
-      BitSets.setAll(childFactors, leftTree.getTreeOrder());
+      childFactorBuilder.addAll(leftTree.getTreeOrder());
     }
-    multiJoin.getChildFactors(rightTree, childFactors);
+    multiJoin.getChildFactors(rightTree, childFactorBuilder);
 
+    final ImmutableBitSet childFactor = childFactorBuilder.build();
     RexNode condition = null;
     final ListIterator<RexNode> filterIter = filtersToAdd.listIterator();
     while (filterIter.hasNext()) {
       RexNode joinFilter = filterIter.next();
-      BitSet filterBitmap =
+      ImmutableBitSet filterBitmap =
           multiJoin.getFactorsRefByJoinFilter(joinFilter);
 
       // if all factors in the join filter are in the join tree,
       // AND the filter to the current join condition
-      if (BitSets.contains(childFactors, filterBitmap)) {
+      if (childFactor.contains(filterBitmap)) {
         if (condition == null) {
           condition = joinFilter;
         } else {
@@ -1837,14 +1847,6 @@ public class LoptOptimizeJoinRule extends RelOptRule {
    * if this is a removable self-join, swap so the factor that should be
    * preserved when the self-join is removed is put on the left.
    *
-   * <p>Note that unlike Broadbase, we do not swap if in the join condition,
-   * the RHS references more columns than the LHS. This can help for queries
-   * like (select * from A,B where A.A between B.X and B.Y). By putting B on
-   * the left, that would result in a sargable predicate with two endpoints.
-   * However, since {@link org.eigenbase.sarg.SargRexAnalyzer} currently
-   * doesn't handle these type of sargable predicates, there's no point in
-   * doing the swap for this reason.
-   *
    * @param multiJoin join factors being optimized
    * @param left left side of join tree
    * @param right right hand side of join tree

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/b9d8de38/core/src/main/java/org/apache/calcite/rel/rules/LoptSemiJoinOptimizer.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/LoptSemiJoinOptimizer.java b/core/src/main/java/org/apache/calcite/rel/rules/LoptSemiJoinOptimizer.java
index 2f95c20..402cd0f 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/LoptSemiJoinOptimizer.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/LoptSemiJoinOptimizer.java
@@ -33,14 +33,13 @@ import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
-import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.ImmutableIntList;
 
 import com.google.common.collect.Lists;
 
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.BitSet;
 import java.util.Comparator;
 import java.util.HashMap;
 import java.util.List;
@@ -209,7 +208,7 @@ public class LoptSemiJoinOptimizer {
     // a join filter and we've already verified that the operands are
     // RexInputRefs, verify that the factors belong to the fact and
     // dimension table
-    BitSet joinRefs = multiJoin.getFactorsRefByJoinFilter(joinFilter);
+    ImmutableBitSet joinRefs = multiJoin.getFactorsRefByJoinFilter(joinFilter);
     assert joinRefs.cardinality() == 2;
     int factor1 = joinRefs.nextSetBit(0);
     int factor2 = joinRefs.nextSetBit(factor1 + 1);
@@ -647,11 +646,8 @@ public class LoptSemiJoinOptimizer {
     // selectivity value is required because of the overhead of
     // index lookups on a very large fact table.  Half was chosen as
     // a middle ground based on testing that was done with a large
-    // dataset.
-    BitSet dimCols = new BitSet();
-    for (int dimCol : semiJoin.getRightKeys()) {
-      dimCols.set(dimCol);
-    }
+    // data set.
+    final ImmutableBitSet dimCols = ImmutableBitSet.of(semiJoin.getRightKeys());
     double selectivity =
         RelMdUtil.computeSemiJoinSelectivity(factRel, dimRel, semiJoin);
     if (selectivity > .5) {
@@ -724,10 +720,7 @@ public class LoptSemiJoinOptimizer {
 
     // Check if the semijoin keys corresponding to the dimension table
     // are unique.  The semijoin will filter out the nulls.
-    BitSet dimKeys = new BitSet();
-    for (Integer key : semiJoin.getRightKeys()) {
-      dimKeys.set(key);
-    }
+    final ImmutableBitSet dimKeys = ImmutableBitSet.of(semiJoin.getRightKeys());
     RelNode dimRel = multiJoin.getJoinFactor(dimIdx);
     if (!RelMdUtil.areColumnsDefinitelyUniqueWhenNullsFiltered(
         dimRel,
@@ -738,12 +731,12 @@ public class LoptSemiJoinOptimizer {
     // check that the only fields referenced from the dimension table
     // in either its projection or join conditions are the dimension
     // keys
-    BitSet dimProjRefs = multiJoin.getProjFields(dimIdx);
+    ImmutableBitSet dimProjRefs = multiJoin.getProjFields(dimIdx);
     if (dimProjRefs == null) {
       int nDimFields = multiJoin.getNumFieldsInJoinFactor(dimIdx);
-      dimProjRefs = BitSets.range(0, nDimFields);
+      dimProjRefs = ImmutableBitSet.range(0, nDimFields);
     }
-    if (!BitSets.contains(dimKeys, dimProjRefs)) {
+    if (!dimKeys.contains(dimProjRefs)) {
       return;
     }
     int [] dimJoinRefCounts = multiJoin.getJoinFieldRefCounts(dimIdx);


[34/58] [abbrv] [partial] incubator-calcite git commit: [CALCITE-306] Standardize code style for "import package.*; "

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/rules/AggregateProjectPullUpConstantsRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AggregateProjectPullUpConstantsRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AggregateProjectPullUpConstantsRule.java
index 6d741f7..cb99e86 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/AggregateProjectPullUpConstantsRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/AggregateProjectPullUpConstantsRule.java
@@ -14,22 +14,36 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
-
-import java.util.*;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.rex.*;
-import org.eigenbase.util.*;
-import org.eigenbase.util.mapping.*;
-
-import net.hydromatic.optiq.util.BitSets;
+package org.apache.calcite.rel.rules;
+
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.logical.LogicalAggregate;
+import org.apache.calcite.rel.logical.LogicalProject;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexLocalRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.IntList;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Permutation;
+import org.apache.calcite.util.mapping.Mapping;
+import org.apache.calcite.util.mapping.MappingType;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
 
 /**
- * PullConstantsThroughAggregatesRule removes constant expressions from the
- * group list of an {@link AggregateRel}.
+ * Planner rule that removes constant expressions from the
+ * group list of an {@link org.apache.calcite.rel.logical.LogicalAggregate}.
  *
  * <p><b>Effect of the rule</b></p>
  *
@@ -38,37 +52,36 @@ import net.hydromatic.optiq.util.BitSets;
  * reduced aggregate. If those constants are not used, another rule will remove
  * them from the project.
  *
- * <p>AggregateRel needs its group columns to be on the prefix of its input
+ * <p>LogicalAggregate needs its group columns to be on the prefix of its input
  * relational expression. Therefore, if a constant is not on the trailing edge
  * of the group list, removing it will leave a hole. In this case, the rule adds
  * a project before the aggregate to reorder the columns, and permutes them back
  * afterwards.
  */
-public class PullConstantsThroughAggregatesRule extends RelOptRule {
+public class AggregateProjectPullUpConstantsRule extends RelOptRule {
   //~ Static fields/initializers ---------------------------------------------
 
   /** The singleton. */
-  public static final PullConstantsThroughAggregatesRule INSTANCE =
-      new PullConstantsThroughAggregatesRule();
+  public static final AggregateProjectPullUpConstantsRule INSTANCE =
+      new AggregateProjectPullUpConstantsRule();
 
   //~ Constructors -----------------------------------------------------------
 
   /**
    * Private: use singleton
    */
-  private PullConstantsThroughAggregatesRule() {
+  private AggregateProjectPullUpConstantsRule() {
     super(
-        operand(
-            AggregateRel.class,
-            operand(ProjectRel.class, any())));
+        operand(LogicalAggregate.class,
+            operand(LogicalProject.class, any())));
   }
 
   //~ Methods ----------------------------------------------------------------
 
   // implement RelOptRule
   public void onMatch(RelOptRuleCall call) {
-    AggregateRel aggregate = call.rel(0);
-    ProjectRel child = call.rel(1);
+    LogicalAggregate aggregate = call.rel(0);
+    LogicalProject child = call.rel(1);
 
     final int groupCount = aggregate.getGroupCount();
     if (groupCount == 1) {
@@ -78,7 +91,7 @@ public class PullConstantsThroughAggregatesRule extends RelOptRule {
     }
 
     final RexProgram program =
-      RexProgram.create(child.getChild().getRowType(),
+      RexProgram.create(child.getInput().getRowType(),
         child.getProjects(), null, child.getRowType(),
         child.getCluster().getRexBuilder());
 
@@ -124,7 +137,7 @@ public class PullConstantsThroughAggregatesRule extends RelOptRule {
                 newGroupCount));
       }
       newAggregate =
-          new AggregateRel(
+          new LogicalAggregate(
               aggregate.getCluster(),
               child,
               BitSets.range(newGroupCount),
@@ -169,7 +182,7 @@ public class PullConstantsThroughAggregatesRule extends RelOptRule {
 
       // Aggregate on projection.
       newAggregate =
-          new AggregateRel(
+          new LogicalAggregate(
               aggregate.getCluster(),
               project,
               BitSets.range(newGroupCount),
@@ -238,4 +251,4 @@ public class PullConstantsThroughAggregatesRule extends RelOptRule {
   }
 }
 
-// End PullConstantsThroughAggregatesRule.java
+// End AggregateProjectPullUpConstantsRule.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/rules/AggregateReduceFunctionsRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AggregateReduceFunctionsRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AggregateReduceFunctionsRule.java
index 78a080a..718063f 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/AggregateReduceFunctionsRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/AggregateReduceFunctionsRule.java
@@ -14,54 +14,93 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
-
-import java.math.BigDecimal;
-import java.util.*;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.rex.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.fun.*;
-import org.eigenbase.sql.type.SqlTypeUtil;
-import org.eigenbase.util.*;
+package org.apache.calcite.rel.rules;
+
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptRuleOperand;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Aggregate;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.logical.LogicalAggregate;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.SqlAggFunction;
+import org.apache.calcite.sql.fun.SqlAvgAggFunction;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.fun.SqlSumAggFunction;
+import org.apache.calcite.sql.type.SqlTypeUtil;
+import org.apache.calcite.util.CompositeList;
+import org.apache.calcite.util.ImmutableIntList;
+import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
 
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
 /**
- * Rule to reduce aggregates to simpler forms. Currently only AVG(x) to
- * SUM(x)/COUNT(x), but eventually will handle others such as STDDEV.
+ * Planner rule that reduces aggregate functions in
+ * {@link org.apache.calcite.rel.core.Aggregate}s to simpler forms.
+ *
+ * <p>Rewrites:
+ * <ul>
+ *
+ * <li>AVG(x) &rarr; SUM(x) / COUNT(x)
+ *
+ * <li>STDDEV_POP(x) &rarr; SQRT(
+ *     (SUM(x * x) - SUM(x) * SUM(x) / COUNT(x))
+ *    / COUNT(x))
+ *
+ * <li>STDDEV_SAMP(x) &rarr; SQRT(
+ *     (SUM(x * x) - SUM(x) * SUM(x) / COUNT(x))
+ *     / CASE COUNT(x) WHEN 1 THEN NULL ELSE COUNT(x) - 1 END)
+ *
+ * <li>VAR_POP(x) &rarr; (SUM(x * x) - SUM(x) * SUM(x) / COUNT(x))
+ *     / COUNT(x)
+ *
+ * <li>VAR_SAMP(x) &rarr; (SUM(x * x) - SUM(x) * SUM(x) / COUNT(x))
+ *        / CASE COUNT(x) WHEN 1 THEN NULL ELSE COUNT(x) - 1 END
+ * </ul>
+ *
+ * <p>Since many of these rewrites introduce multiple occurrences of simpler
+ * forms like {@code COUNT(x)}, the rule gathers common sub-expressions as it
+ * goes.
  */
-public class ReduceAggregatesRule extends RelOptRule {
+public class AggregateReduceFunctionsRule extends RelOptRule {
   //~ Static fields/initializers ---------------------------------------------
 
-  /**
-   * The singleton.
-   */
-  public static final ReduceAggregatesRule INSTANCE =
-      new ReduceAggregatesRule(operand(AggregateRel.class, any()));
+  /** The singleton. */
+  public static final AggregateReduceFunctionsRule INSTANCE =
+      new AggregateReduceFunctionsRule(operand(LogicalAggregate.class, any()));
 
   //~ Constructors -----------------------------------------------------------
 
-  protected ReduceAggregatesRule(RelOptRuleOperand operand) {
+  protected AggregateReduceFunctionsRule(RelOptRuleOperand operand) {
     super(operand);
   }
 
   //~ Methods ----------------------------------------------------------------
 
-  @Override
-  public boolean matches(RelOptRuleCall call) {
+  @Override public boolean matches(RelOptRuleCall call) {
     if (!super.matches(call)) {
       return false;
     }
-    AggregateRelBase oldAggRel = (AggregateRelBase) call.rels[0];
+    Aggregate oldAggRel = (Aggregate) call.rels[0];
     return containsAvgStddevVarCall(oldAggRel.getAggCallList());
   }
 
   public void onMatch(RelOptRuleCall ruleCall) {
-    AggregateRelBase oldAggRel = (AggregateRelBase) ruleCall.rels[0];
+    Aggregate oldAggRel = (Aggregate) ruleCall.rels[0];
     reduceAggs(ruleCall, oldAggRel);
   }
 
@@ -89,7 +128,7 @@ public class ReduceAggregatesRule extends RelOptRule {
    */
   private void reduceAggs(
       RelOptRuleCall ruleCall,
-      AggregateRelBase oldAggRel) {
+      Aggregate oldAggRel) {
     RexBuilder rexBuilder = oldAggRel.getCluster().getRexBuilder();
 
     List<AggregateCall> oldCalls = oldAggRel.getAggCallList();
@@ -112,7 +151,7 @@ public class ReduceAggregatesRule extends RelOptRule {
     // List of input expressions. If a particular aggregate needs more, it
     // will add an expression to the end, and we will create an extra
     // project.
-    RelNode input = oldAggRel.getChild();
+    RelNode input = oldAggRel.getInput();
     List<RexNode> inputExprs = new ArrayList<RexNode>();
     for (RelDataTypeField field : input.getRowType().getFieldList()) {
       inputExprs.add(
@@ -140,7 +179,7 @@ public class ReduceAggregatesRule extends RelOptRule {
                       extraArgCount,
                       null)));
     }
-    AggregateRelBase newAggRel =
+    Aggregate newAggRel =
         newAggregateRel(
             oldAggRel, input, newCalls);
 
@@ -159,7 +198,7 @@ public class ReduceAggregatesRule extends RelOptRule {
   }
 
   private RexNode reduceAgg(
-      AggregateRelBase oldAggRel,
+      Aggregate oldAggRel,
       AggregateCall oldCall,
       List<AggregateCall> newCalls,
       Map<AggregateCall, RexNode> aggCallMapping,
@@ -226,7 +265,7 @@ public class ReduceAggregatesRule extends RelOptRule {
   }
 
   private RexNode reduceAvg(
-      AggregateRelBase oldAggRel,
+      Aggregate oldAggRel,
       AggregateCall oldCall,
       List<AggregateCall> newCalls,
       Map<AggregateCall, RexNode> aggCallMapping) {
@@ -237,7 +276,7 @@ public class ReduceAggregatesRule extends RelOptRule {
     int iAvgInput = oldCall.getArgList().get(0);
     RelDataType avgInputType =
         getFieldType(
-            oldAggRel.getChild(),
+            oldAggRel.getInput(),
             iAvgInput);
     RelDataType sumType =
         typeFactory.createTypeWithNullability(
@@ -257,7 +296,7 @@ public class ReduceAggregatesRule extends RelOptRule {
             oldCall.isDistinct(),
             oldCall.getArgList(),
             oldAggRel.getGroupCount(),
-            oldAggRel.getChild(),
+            oldAggRel.getInput(),
             null,
             null);
 
@@ -287,7 +326,7 @@ public class ReduceAggregatesRule extends RelOptRule {
   }
 
   private RexNode reduceSum(
-      AggregateRelBase oldAggRel,
+      Aggregate oldAggRel,
       AggregateCall oldCall,
       List<AggregateCall> newCalls,
       Map<AggregateCall, RexNode> aggCallMapping) {
@@ -298,7 +337,7 @@ public class ReduceAggregatesRule extends RelOptRule {
     int arg = oldCall.getArgList().get(0);
     RelDataType argType =
         getFieldType(
-            oldAggRel.getChild(),
+            oldAggRel.getInput(),
             arg);
     final RelDataType sumType =
         typeFactory.createTypeWithNullability(
@@ -350,7 +389,7 @@ public class ReduceAggregatesRule extends RelOptRule {
   }
 
   private RexNode reduceStddev(
-      AggregateRelBase oldAggRel,
+      Aggregate oldAggRel,
       AggregateCall oldCall,
       boolean biased,
       boolean sqrt,
@@ -377,7 +416,7 @@ public class ReduceAggregatesRule extends RelOptRule {
     final int argOrdinal = oldCall.getArgList().get(0);
     final RelDataType argType =
         getFieldType(
-            oldAggRel.getChild(),
+            oldAggRel.getInput(),
             argOrdinal);
 
     final RexNode argRef = inputExprs.get(argOrdinal);
@@ -430,7 +469,7 @@ public class ReduceAggregatesRule extends RelOptRule {
             oldCall.isDistinct(),
             oldCall.getArgList(),
             oldAggRel.getGroupCount(),
-            oldAggRel.getChild(),
+            oldAggRel.getInput(),
             null,
             null);
     final RexNode countArg =
@@ -507,19 +546,19 @@ public class ReduceAggregatesRule extends RelOptRule {
 
   /**
    * Do a shallow clone of oldAggRel and update aggCalls. Could be refactored
-   * into AggregateRelBase and subclasses - but it's only needed for some
+   * into Aggregate and subclasses - but it's only needed for some
    * subclasses.
    *
-   * @param oldAggRel AggregateRel to clone.
+   * @param oldAggRel LogicalAggregate to clone.
    * @param inputRel  Input relational expression
    * @param newCalls  New list of AggregateCalls
    * @return shallow clone with new list of AggregateCalls.
    */
-  protected AggregateRelBase newAggregateRel(
-      AggregateRelBase oldAggRel,
+  protected Aggregate newAggregateRel(
+      Aggregate oldAggRel,
       RelNode inputRel,
       List<AggregateCall> newCalls) {
-    return new AggregateRel(
+    return new LogicalAggregate(
         oldAggRel.getCluster(),
         inputRel,
         oldAggRel.getGroupSet(),
@@ -533,4 +572,4 @@ public class ReduceAggregatesRule extends RelOptRule {
   }
 }
 
-// End ReduceAggregatesRule.java
+// End AggregateReduceFunctionsRule.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/rules/AggregateRemoveRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AggregateRemoveRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AggregateRemoveRule.java
index 970bded..5c11012 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/AggregateRemoveRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/AggregateRemoveRule.java
@@ -14,41 +14,45 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
+package org.apache.calcite.rel.rules;
 
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.logical.LogicalAggregate;
 
 /**
- * Rule to remove an {@link AggregateRel} implementing DISTINCT if the
- * underlying relational expression is already distinct.
+ * Planner rule that removes
+ * a {@link org.apache.calcite.rel.logical.LogicalAggregate}
+ * if it computes no aggregate functions
+ * (that is, it is implementing {@code SELECT DISTINCT})
+ * and the underlying relational expression is already distinct.
  */
-public class RemoveDistinctRule extends RelOptRule {
-  public static final RemoveDistinctRule INSTANCE =
-      new RemoveDistinctRule();
+public class AggregateRemoveRule extends RelOptRule {
+  public static final AggregateRemoveRule INSTANCE =
+      new AggregateRemoveRule();
 
   //~ Constructors -----------------------------------------------------------
 
   /**
-   * Creates a RemoveDistinctRule.
+   * Creates a AggregateRemoveRule.
    */
-  private RemoveDistinctRule() {
+  private AggregateRemoveRule() {
     // REVIEW jvs 14-Mar-2006: We have to explicitly mention the child here
     // to make sure the rule re-fires after the child changes (e.g. via
-    // RemoveTrivialProjectRule), since that may change our information
+    // ProjectRemoveRule), since that may change our information
     // about whether the child is distinct.  If we clean up the inference of
     // distinct to make it correct up-front, we can get rid of the reference
     // to the child here.
     super(
-        operand(
-            AggregateRel.class,
+        operand(LogicalAggregate.class,
             operand(RelNode.class, any())));
   }
 
   //~ Methods ----------------------------------------------------------------
 
   public void onMatch(RelOptRuleCall call) {
-    AggregateRel distinct = call.rel(0);
+    LogicalAggregate distinct = call.rel(0);
     RelNode child = call.rel(1);
     if (!distinct.getAggCallList().isEmpty()
         || !child.isKey(distinct.getGroupSet())) {
@@ -65,4 +69,4 @@ public class RemoveDistinctRule extends RelOptRule {
   }
 }
 
-// End RemoveDistinctRule.java
+// End AggregateRemoveRule.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/rules/AggregateStarTableRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AggregateStarTableRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AggregateStarTableRule.java
index 3aeca5a..d511f79 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/AggregateStarTableRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/AggregateStarTableRule.java
@@ -14,43 +14,42 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
+package org.apache.calcite.rel.rules;
 
-import java.util.BitSet;
-import java.util.List;
-
-import org.eigenbase.rel.AggregateCall;
-import org.eigenbase.rel.AggregateRelBase;
-import org.eigenbase.rel.ProjectRelBase;
-import org.eigenbase.rel.RelNode;
-import org.eigenbase.relopt.RelOptCluster;
-import org.eigenbase.relopt.RelOptLattice;
-import org.eigenbase.relopt.RelOptRule;
-import org.eigenbase.relopt.RelOptRuleCall;
-import org.eigenbase.relopt.RelOptRuleOperand;
-import org.eigenbase.relopt.RelOptTable;
-import org.eigenbase.relopt.RelOptUtil;
-import org.eigenbase.relopt.SubstitutionVisitor;
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.sql.SqlAggFunction;
-import org.eigenbase.util.Pair;
-import org.eigenbase.util.mapping.AbstractSourceMapping;
-
-import net.hydromatic.optiq.Table;
-import net.hydromatic.optiq.impl.StarTable;
-import net.hydromatic.optiq.jdbc.OptiqSchema;
-import net.hydromatic.optiq.materialize.Lattice;
-import net.hydromatic.optiq.materialize.TileKey;
-import net.hydromatic.optiq.prepare.OptiqPrepareImpl;
-import net.hydromatic.optiq.prepare.RelOptTableImpl;
-import net.hydromatic.optiq.util.BitSets;
+import org.apache.calcite.jdbc.CalciteSchema;
+import org.apache.calcite.materialize.Lattice;
+import org.apache.calcite.materialize.TileKey;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptLattice;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptRuleOperand;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.SubstitutionVisitor;
+import org.apache.calcite.prepare.CalcitePrepareImpl;
+import org.apache.calcite.prepare.RelOptTableImpl;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Aggregate;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.schema.Table;
+import org.apache.calcite.schema.impl.StarTable;
+import org.apache.calcite.sql.SqlAggFunction;
+import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.mapping.AbstractSourceMapping;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 
+import java.util.BitSet;
+import java.util.List;
+
 /**
- * Planner rule that matches an {@link org.eigenbase.rel.AggregateRelBase} on
- * top of a {@link net.hydromatic.optiq.impl.StarTable.StarTableScan}.
+ * Planner rule that matches an {@link org.apache.calcite.rel.core.Aggregate} on
+ * top of a {@link org.apache.calcite.schema.impl.StarTable.StarTableScan}.
  *
  * <p>This pattern indicates that an aggregate table may exist. The rule asks
  * the star table for an aggregate table at the required level of aggregation.
@@ -58,31 +57,30 @@ import com.google.common.collect.Lists;
 public class AggregateStarTableRule extends RelOptRule {
   public static final AggregateStarTableRule INSTANCE =
       new AggregateStarTableRule(
-          operand(AggregateRelBase.class,
+          operand(Aggregate.class,
               some(operand(StarTable.StarTableScan.class, none()))),
           "AggregateStarTableRule");
 
   public static final AggregateStarTableRule INSTANCE2 =
       new AggregateStarTableRule(
-          operand(AggregateRelBase.class,
-              operand(ProjectRelBase.class,
+          operand(Aggregate.class,
+              operand(Project.class,
                   operand(StarTable.StarTableScan.class, none()))),
           "AggregateStarTableRule:project") {
-        @Override
-        public void onMatch(RelOptRuleCall call) {
-          final AggregateRelBase aggregate = call.rel(0);
-          final ProjectRelBase project = call.rel(1);
+        @Override public void onMatch(RelOptRuleCall call) {
+          final Aggregate aggregate = call.rel(0);
+          final Project project = call.rel(1);
           final StarTable.StarTableScan scan = call.rel(2);
           final RelNode rel =
               AggregateProjectMergeRule.apply(aggregate, project);
-          final AggregateRelBase aggregate2;
-          final ProjectRelBase project2;
-          if (rel instanceof AggregateRelBase) {
+          final Aggregate aggregate2;
+          final Project project2;
+          if (rel instanceof Aggregate) {
             project2 = null;
-            aggregate2 = (AggregateRelBase) rel;
-          } else if (rel instanceof ProjectRelBase) {
-            project2 = (ProjectRelBase) rel;
-            aggregate2 = (AggregateRelBase) project2.getChild();
+            aggregate2 = (Aggregate) rel;
+          } else if (rel instanceof Project) {
+            project2 = (Project) rel;
+            aggregate2 = (Aggregate) project2.getInput();
           } else {
             return;
           }
@@ -95,27 +93,26 @@ public class AggregateStarTableRule extends RelOptRule {
     super(operand, description);
   }
 
-  @Override
-  public void onMatch(RelOptRuleCall call) {
-    final AggregateRelBase aggregate = call.rel(0);
+  @Override public void onMatch(RelOptRuleCall call) {
+    final Aggregate aggregate = call.rel(0);
     final StarTable.StarTableScan scan = call.rel(1);
     apply(call, null, aggregate, scan);
   }
 
-  protected void apply(RelOptRuleCall call, ProjectRelBase postProject,
-      final AggregateRelBase aggregate, StarTable.StarTableScan scan) {
+  protected void apply(RelOptRuleCall call, Project postProject,
+      final Aggregate aggregate, StarTable.StarTableScan scan) {
     final RelOptCluster cluster = scan.getCluster();
     final RelOptTable table = scan.getTable();
     final RelOptLattice lattice = call.getPlanner().getLattice(table);
     final List<Lattice.Measure> measures =
         lattice.lattice.toMeasures(aggregate.getAggCallList());
-    final Pair<OptiqSchema.TableEntry, TileKey> pair =
+    final Pair<CalciteSchema.TableEntry, TileKey> pair =
         lattice.getAggregate(call.getPlanner(), aggregate.getGroupSet(),
             measures);
     if (pair == null) {
       return;
     }
-    final OptiqSchema.TableEntry tableEntry = pair.left;
+    final CalciteSchema.TableEntry tableEntry = pair.left;
     final TileKey tileKey = pair.right;
     final double rowCount = aggregate.getRows();
     final Table aggregateTable = tableEntry.getTable();
@@ -126,14 +123,14 @@ public class AggregateStarTableRule extends RelOptRule {
             tableEntry, rowCount);
     RelNode rel = aggregateRelOptTable.toRel(RelOptUtil.getContext(cluster));
     if (tileKey == null) {
-      if (OptiqPrepareImpl.DEBUG) {
+      if (CalcitePrepareImpl.DEBUG) {
         System.out.println("Using materialization "
             + aggregateRelOptTable.getQualifiedName()
             + " (exact match)");
       }
     } else if (!tileKey.dimensions.equals(aggregate.getGroupSet())) {
       // Aggregate has finer granularity than we need. Roll up.
-      if (OptiqPrepareImpl.DEBUG) {
+      if (CalcitePrepareImpl.DEBUG) {
         System.out.println("Using materialization "
             + aggregateRelOptTable.getQualifiedName()
             + ", rolling up " + tileKey.dimensions + " to "

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/rules/AggregateUnionAggregateRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AggregateUnionAggregateRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AggregateUnionAggregateRule.java
index e7819f9..f20cd69 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/AggregateUnionAggregateRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/AggregateUnionAggregateRule.java
@@ -14,39 +14,42 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
+package org.apache.calcite.rel.rules;
 
-import java.util.List;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.logical.LogicalAggregate;
+import org.apache.calcite.rel.logical.LogicalUnion;
 
 import com.google.common.collect.ImmutableList;
 
+import java.util.List;
+
 /**
- * PullUpAggregateAboveUnionRule implements the rule for pulling {@link
- * AggregateRel}s beneath a {@link UnionRel} so two {@link AggregateRel}s that
- * are used to remove duplicates can be combined into a single {@link
- * AggregateRel}.
+ * Planner rule that matches
+ * {@link org.apache.calcite.rel.logical.LogicalAggregate}s beneath a
+ * {@link org.apache.calcite.rel.logical.LogicalUnion} and pulls them up, so
+ * that a single
+ * {@link org.apache.calcite.rel.logical.LogicalAggregate} removes duplicates.
  *
- * <p>This rule only handles cases where the {@link UnionRel}s still have only
- * two inputs.
+ * <p>This rule only handles cases where the
+ * {@link org.apache.calcite.rel.logical.LogicalUnion}s
+ * still have only two inputs.
  */
-public class PullUpAggregateAboveUnionRule extends RelOptRule {
-  public static final PullUpAggregateAboveUnionRule INSTANCE =
-      new PullUpAggregateAboveUnionRule();
+public class AggregateUnionAggregateRule extends RelOptRule {
+  public static final AggregateUnionAggregateRule INSTANCE =
+      new AggregateUnionAggregateRule();
 
   //~ Constructors -----------------------------------------------------------
 
   /**
-   * Creates a PullUpAggregateAboveUnionRule.
+   * Creates a AggregateUnionAggregateRule.
    */
-  private PullUpAggregateAboveUnionRule() {
+  private AggregateUnionAggregateRule() {
     super(
-        operand(
-            AggregateRel.class,
-            operand(
-                UnionRel.class,
+        operand(LogicalAggregate.class,
+            operand(LogicalUnion.class,
                 operand(RelNode.class, any()),
                 operand(RelNode.class, any()))));
   }
@@ -55,28 +58,28 @@ public class PullUpAggregateAboveUnionRule extends RelOptRule {
 
   // implement RelOptRule
   public void onMatch(RelOptRuleCall call) {
-    UnionRel unionRel = call.rel(1);
+    LogicalUnion union = call.rel(1);
 
     // If distincts haven't been removed yet, defer invoking this rule
-    if (!unionRel.all) {
+    if (!union.all) {
       return;
     }
 
-    AggregateRel topAggRel = call.rel(0);
-    AggregateRel bottomAggRel;
+    LogicalAggregate topAggRel = call.rel(0);
+    LogicalAggregate bottomAggRel;
 
-    // We want to apply this rule on the pattern where the AggregateRel
-    // is the second input into the UnionRel first.  Hence, that's why the
+    // We want to apply this rule on the pattern where the LogicalAggregate
+    // is the second input into the Union first.  Hence, that's why the
     // rule pattern matches on generic RelNodes rather than explicit
     // UnionRels.  By doing so, and firing this rule in a bottom-up order,
     // it allows us to only specify a single pattern for this rule.
     List<RelNode> unionInputs;
-    if (call.rel(3) instanceof AggregateRel) {
+    if (call.rel(3) instanceof LogicalAggregate) {
       bottomAggRel = call.rel(3);
       unionInputs = ImmutableList.of(
           call.rel(2),
           call.rel(3).getInput(0));
-    } else if (call.rel(2) instanceof AggregateRel) {
+    } else if (call.rel(2) instanceof LogicalAggregate) {
       bottomAggRel = call.rel(2);
       unionInputs = ImmutableList.of(
           call.rel(2).getInput(0),
@@ -91,16 +94,16 @@ public class PullUpAggregateAboveUnionRule extends RelOptRule {
       return;
     }
 
-    UnionRel newUnionRel =
-        new UnionRel(
-            unionRel.getCluster(),
+    LogicalUnion newUnion =
+        new LogicalUnion(
+            union.getCluster(),
             unionInputs,
             true);
 
-    AggregateRel newAggRel =
-        new AggregateRel(
+    LogicalAggregate newAggRel =
+        new LogicalAggregate(
             topAggRel.getCluster(),
-            newUnionRel,
+            newUnion,
             topAggRel.getGroupSet(),
             topAggRel.getAggCallList());
 
@@ -108,4 +111,4 @@ public class PullUpAggregateAboveUnionRule extends RelOptRule {
   }
 }
 
-// End PullUpAggregateAboveUnionRule.java
+// End AggregateUnionAggregateRule.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/rules/AggregateUnionTransposeRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AggregateUnionTransposeRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AggregateUnionTransposeRule.java
index 6d8796f..7b92704 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/AggregateUnionTransposeRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/AggregateUnionTransposeRule.java
@@ -14,29 +14,42 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
-
-import java.util.*;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.rel.metadata.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.sql.SqlAggFunction;
-import org.eigenbase.sql.fun.*;
-
-import net.hydromatic.linq4j.Ord;
+package org.apache.calcite.rel.rules;
+
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.logical.LogicalAggregate;
+import org.apache.calcite.rel.logical.LogicalUnion;
+import org.apache.calcite.rel.metadata.RelMdUtil;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlAggFunction;
+import org.apache.calcite.sql.fun.SqlCountAggFunction;
+import org.apache.calcite.sql.fun.SqlMinMaxAggFunction;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.fun.SqlSumAggFunction;
+import org.apache.calcite.sql.fun.SqlSumEmptyIsZeroAggFunction;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 
+import java.util.ArrayList;
+import java.util.IdentityHashMap;
+import java.util.List;
+import java.util.Map;
+
 /**
- * PushAggregateThroughUnionRule implements the rule for pushing an
- * {@link AggregateRel} past a non-distinct {@link UnionRel}.
+ * Planner rule that pushes an
+ * {@link org.apache.calcite.rel.logical.LogicalAggregate}
+ * past a non-distinct {@link org.apache.calcite.rel.logical.LogicalUnion}.
  */
-public class PushAggregateThroughUnionRule extends RelOptRule {
-  public static final PushAggregateThroughUnionRule INSTANCE =
-      new PushAggregateThroughUnionRule();
+public class AggregateUnionTransposeRule extends RelOptRule {
+  public static final AggregateUnionTransposeRule INSTANCE =
+      new AggregateUnionTransposeRule();
 
   private static final Map<Class, Boolean> SUPPORTED_AGGREGATES =
       new IdentityHashMap<Class, Boolean>();
@@ -51,18 +64,17 @@ public class PushAggregateThroughUnionRule extends RelOptRule {
   /**
    * Private constructor.
    */
-  private PushAggregateThroughUnionRule() {
+  private AggregateUnionTransposeRule() {
     super(
-        operand(
-            AggregateRel.class,
-            operand(UnionRel.class, any())));
+        operand(LogicalAggregate.class,
+            operand(LogicalUnion.class, any())));
   }
 
   public void onMatch(RelOptRuleCall call) {
-    AggregateRel aggRel = call.rel(0);
-    UnionRel unionRel = call.rel(1);
+    LogicalAggregate aggRel = call.rel(0);
+    LogicalUnion union = call.rel(1);
 
-    if (!unionRel.all) {
+    if (!union.all) {
       // This transformation is only valid for UNION ALL.
       // Consider t1(i) with rows (5), (5) and t2(i) with
       // rows (5), (10), and the query
@@ -75,7 +87,7 @@ public class PushAggregateThroughUnionRule extends RelOptRule {
       return;
     }
 
-    RelOptCluster cluster = unionRel.getCluster();
+    RelOptCluster cluster = union.getCluster();
 
     List<AggregateCall> transformedAggCalls =
         transformAggCalls(aggRel,
@@ -91,7 +103,7 @@ public class PushAggregateThroughUnionRule extends RelOptRule {
 
     // create corresponding aggs on top of each union child
     List<RelNode> newUnionInputs = new ArrayList<RelNode>();
-    for (RelNode input : unionRel.getInputs()) {
+    for (RelNode input : union.getInputs()) {
       boolean alreadyUnique =
           RelMdUtil.areColumnsDefinitelyUnique(
               input,
@@ -102,7 +114,7 @@ public class PushAggregateThroughUnionRule extends RelOptRule {
       } else {
         anyTransformed = true;
         newUnionInputs.add(
-            new AggregateRel(
+            new LogicalAggregate(
                 cluster, input,
                 aggRel.getGroupSet(),
                 aggRel.getAggCallList()));
@@ -117,11 +129,11 @@ public class PushAggregateThroughUnionRule extends RelOptRule {
     }
 
     // create a new union whose children are the aggs created above
-    UnionRel newUnionRel = new UnionRel(cluster, newUnionInputs, true);
+    LogicalUnion newUnion = new LogicalUnion(cluster, newUnionInputs, true);
 
-    AggregateRel newTopAggRel = new AggregateRel(
+    LogicalAggregate newTopAggRel = new LogicalAggregate(
         cluster,
-        newUnionRel,
+        newUnion,
         aggRel.getGroupSet(),
         transformedAggCalls);
 
@@ -169,4 +181,4 @@ public class PushAggregateThroughUnionRule extends RelOptRule {
   }
 }
 
-// End PushAggregateThroughUnionRule.java
+// End AggregateUnionTransposeRule.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/rules/CalcMergeRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/CalcMergeRule.java b/core/src/main/java/org/apache/calcite/rel/rules/CalcMergeRule.java
index 79a82b1..0b749d3 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/CalcMergeRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/CalcMergeRule.java
@@ -14,36 +14,44 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
+package org.apache.calcite.rel.rules;
 
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.rex.*;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.rel.core.Calc;
+import org.apache.calcite.rex.RexOver;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexProgramBuilder;
 
 /**
- * Planner rule which merges a {@link CalcRel} onto a {@link CalcRel}. The
- * resulting {@link CalcRel} has the same project list as the upper {@link
- * CalcRel}, but expressed in terms of the lower {@link CalcRel}'s inputs.
+ * Planner rule that merges a
+ * {@link org.apache.calcite.rel.logical.LogicalCalc} onto a
+ * {@link org.apache.calcite.rel.logical.LogicalCalc}.
+ *
+ * <p>The resulting {@link org.apache.calcite.rel.logical.LogicalCalc} has the
+ * same project list as the upper
+ * {@link org.apache.calcite.rel.logical.LogicalCalc}, but expressed in terms of
+ * the lower {@link org.apache.calcite.rel.logical.LogicalCalc}'s inputs.
  */
-public class MergeCalcRule extends RelOptRule {
+public class CalcMergeRule extends RelOptRule {
   //~ Static fields/initializers ---------------------------------------------
 
-  public static final MergeCalcRule INSTANCE = new MergeCalcRule();
+  public static final CalcMergeRule INSTANCE = new CalcMergeRule();
 
   //~ Constructors -----------------------------------------------------------
 
-  private MergeCalcRule() {
+  private CalcMergeRule() {
     super(
         operand(
-            CalcRelBase.class,
-            operand(CalcRelBase.class, any())));
+            Calc.class,
+            operand(Calc.class, any())));
   }
 
   //~ Methods ----------------------------------------------------------------
 
   public void onMatch(RelOptRuleCall call) {
-    final CalcRelBase topCalc = call.rel(0);
-    final CalcRelBase bottomCalc = call.rel(1);
+    final Calc topCalc = call.rel(0);
+    final Calc bottomCalc = call.rel(1);
 
     // Don't merge a calc which contains windowed aggregates onto a
     // calc. That would effectively be pushing a windowed aggregate down
@@ -62,10 +70,10 @@ public class MergeCalcRule extends RelOptRule {
             topCalc.getCluster().getRexBuilder());
     assert mergedProgram.getOutputRowType()
         == topProgram.getOutputRowType();
-    final CalcRelBase newCalc =
+    final Calc newCalc =
         topCalc.copy(
             topCalc.getTraitSet(),
-            bottomCalc.getChild(),
+            bottomCalc.getInput(),
             mergedProgram,
             topCalc.getCollationList());
 
@@ -79,4 +87,4 @@ public class MergeCalcRule extends RelOptRule {
   }
 }
 
-// End MergeCalcRule.java
+// End CalcMergeRule.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/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 faefefa..5797273 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
@@ -14,38 +14,63 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
-
-import java.io.*;
-
-import java.util.*;
-import java.util.logging.*;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.rex.*;
-import org.eigenbase.util.*;
-
-import net.hydromatic.optiq.util.graph.*;
+package org.apache.calcite.rel.rules;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Calc;
+import org.apache.calcite.rel.logical.LogicalCalc;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexDynamicParam;
+import org.apache.calcite.rex.RexFieldAccess;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexLocalRef;
+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.rex.RexVisitorImpl;
+import org.apache.calcite.util.IntList;
+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 java.io.PrintWriter;
+import java.io.StringWriter;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.logging.Level;
+import java.util.logging.Logger;
 
 /**
- * CalcRelSplitter operates on a {@link CalcRel} with multiple {@link RexCall}
- * sub-expressions that cannot all be implemented by a single concrete {@link
- * RelNode}.
+ * CalcRelSplitter operates on a
+ * {@link org.apache.calcite.rel.core.Calc} with multiple {@link RexCall}
+ * sub-expressions that cannot all be implemented by a single concrete
+ * {@link RelNode}.
  *
  * <p>For example, the Java and Fennel calculator do not implement an identical
- * set of operators. The CalcRel can be used to split a single CalcRel with
- * mixed Java- and Fennel-only operators into a tree of CalcRel object that can
+ * set of operators. The Calc can be used to split a single Calc with
+ * mixed Java- and Fennel-only operators into a tree of Calc object that can
  * each be individually implemented by either Java or Fennel.and splits it into
- * several CalcRel instances.
+ * several Calc instances.
  *
  * <p>Currently the splitter is only capable of handling two "rel types". That
- * is, it can deal with Java vs. Fennel CalcRels, but not Java vs. Fennel vs.
- * some other type of CalcRel.
+ * is, it can deal with Java vs. Fennel Calcs, but not Java vs. Fennel vs.
+ * some other type of Calc.
  *
- * <p>See {@link org.eigenbase.rel.rules.WindowedAggSplitterRule} for an example
- * of how this class is used.
+ * <p>See {@link ProjectToWindowRule}
+ * for an example of how this class is used.
  */
 public abstract class CalcRelSplitter {
   //~ Static fields/initializers ---------------------------------------------
@@ -67,11 +92,11 @@ public abstract class CalcRelSplitter {
   /**
    * Constructs a CalcRelSplitter.
    *
-   * @param calc     CalcRel to split
+   * @param calc     Calc to split
    * @param relTypes Array of rel types, e.g. {Java, Fennel}. Must be
    *                 distinct.
    */
-  CalcRelSplitter(CalcRelBase calc, RelType[] relTypes) {
+  CalcRelSplitter(Calc calc, RelType[] relTypes) {
     for (int i = 0; i < relTypes.length; i++) {
       assert relTypes[i] != null;
       for (int j = 0; j < i; j++) {
@@ -83,7 +108,7 @@ public abstract class CalcRelSplitter {
     this.cluster = calc.getCluster();
     this.traits = calc.getTraitSet();
     this.typeFactory = calc.getCluster().getTypeFactory();
-    this.child = calc.getChild();
+    this.child = calc.getInput();
     this.relTypes = relTypes;
   }
 
@@ -102,7 +127,7 @@ public abstract class CalcRelSplitter {
     // Figure out what level each expression belongs to.
     int[] exprLevels = new int[exprs.length];
 
-    // The reltype of a level is given by
+    // The type of a level is given by
     // relTypes[levelTypeOrdinals[level]].
     int[] levelTypeOrdinals = new int[exprs.length];
 
@@ -205,8 +230,8 @@ public abstract class CalcRelSplitter {
 
       // Sometimes a level's program merely projects its inputs. We don't
       // want these. They cause an explosion in the search space.
-      if (rel instanceof CalcRel
-          && ((CalcRel) rel).getProgram().isTrivial()) {
+      if (rel instanceof LogicalCalc
+          && ((LogicalCalc) rel).getProgram().isTrivial()) {
         rel = rel.getInput(0);
       }
 
@@ -295,7 +320,7 @@ public abstract class CalcRelSplitter {
     levelLoop:
       for (;; ++level) {
         if (level >= levelCount) {
-          // This is a new level. We can use any reltype we like.
+          // This is a new level. We can use any type we like.
           for (int relTypeOrdinal = 0;
               relTypeOrdinal < relTypes.length;
               relTypeOrdinal++) {
@@ -305,7 +330,7 @@ public abstract class CalcRelSplitter {
             if (relTypes[relTypeOrdinal].canImplement(
                 expr,
                 condition)) {
-              // Success. We have found a reltype where we can
+              // Success. We have found a type where we can
               // implement this expression.
               exprLevels[i] = level;
               levelTypeOrdinals[level] = relTypeOrdinal;
@@ -346,7 +371,7 @@ public abstract class CalcRelSplitter {
           // implement expr. But maybe we could succeed with a new
           // level, with all options open?
           if (count(relTypesPossibleForTopLevel) >= relTypes.length) {
-            // Cannot implement for any reltype.
+            // Cannot implement for any type.
             throw Util.newInternal("cannot implement " + expr);
           }
           levelTypeOrdinals[levelCount] =
@@ -358,7 +383,7 @@ public abstract class CalcRelSplitter {
           if (!relTypes[levelTypeOrdinal].canImplement(
               expr,
               condition)) {
-            // Cannot implement this expression in this reltype;
+            // Cannot implement this expression in this type;
             // continue to next level.
             continue;
           }
@@ -603,7 +628,7 @@ public abstract class CalcRelSplitter {
    *
    * @param exprs             Array expressions
    * @param exprLevels        For each expression, the ordinal of its level
-   * @param levelTypeOrdinals For each level, the ordinal of its reltype in
+   * @param levelTypeOrdinals For each level, the ordinal of its type in
    *                          the {@link #relTypes} array
    * @param levelCount        The number of levels
    */
@@ -618,8 +643,7 @@ public abstract class CalcRelSplitter {
     traceWriter.println(program.toString());
 
     for (int level = 0; level < levelCount; level++) {
-      traceWriter.println(
-          "Rel Level " + level
+      traceWriter.println("Rel Level " + level
           + ", type " + relTypes[levelTypeOrdinals[level]]);
 
       for (int i = 0; i < exprs.length; i++) {
@@ -686,13 +710,13 @@ public abstract class CalcRelSplitter {
    * @param relTypeName Name of a {@link RelType}
    * @return Whether relational expression can be implemented
    */
-  protected boolean canImplement(CalcRel rel, String relTypeName) {
+  protected boolean canImplement(LogicalCalc rel, String relTypeName) {
     for (RelType relType : relTypes) {
       if (relType.name.equals(relTypeName)) {
         return relType.canImplement(rel.getProgram());
       }
     }
-    throw Util.newInternal("unknown reltype " + relTypeName);
+    throw Util.newInternal("unknown type " + relTypeName);
   }
 
   /**
@@ -752,7 +776,7 @@ public abstract class CalcRelSplitter {
         RelDataType rowType,
         RelNode child,
         RexProgram program) {
-      return new CalcRel(
+      return new LogicalCalc(
           cluster,
           traits,
           child,

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/rules/CalcRemoveRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/CalcRemoveRule.java b/core/src/main/java/org/apache/calcite/rel/rules/CalcRemoveRule.java
index c663105..879b8f2 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/CalcRemoveRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/CalcRemoveRule.java
@@ -14,37 +14,41 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
+package org.apache.calcite.rel.rules;
 
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.rex.*;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.logical.LogicalCalc;
+import org.apache.calcite.rex.RexProgram;
 
 /**
- * Rule which removes a trivial {@link CalcRel}.
+ * Planner rule that removes a trivial
+ * {@link org.apache.calcite.rel.logical.LogicalCalc}.
  *
- * <p>A {@link CalcRel} is trivial if it projects its input fields in their
+ * <p>A {@link org.apache.calcite.rel.logical.LogicalCalc}
+ * is trivial if it projects its input fields in their
  * original order, and it does not filter.
  *
- * @see org.eigenbase.rel.rules.RemoveTrivialProjectRule
+ * @see ProjectRemoveRule
  */
-public class RemoveTrivialCalcRule extends RelOptRule {
+public class CalcRemoveRule extends RelOptRule {
   //~ Static fields/initializers ---------------------------------------------
 
-  public static final RemoveTrivialCalcRule INSTANCE =
-      new RemoveTrivialCalcRule();
+  public static final CalcRemoveRule INSTANCE =
+      new CalcRemoveRule();
 
   //~ Constructors -----------------------------------------------------------
 
-  private RemoveTrivialCalcRule() {
-    super(operand(CalcRel.class, any()));
+  private CalcRemoveRule() {
+    super(operand(LogicalCalc.class, any()));
   }
 
   //~ Methods ----------------------------------------------------------------
 
   // implement RelOptRule
   public void onMatch(RelOptRuleCall call) {
-    CalcRel calc = call.rel(0);
+    LogicalCalc calc = call.rel(0);
     RexProgram program = calc.getProgram();
     if (!program.isTrivial()) {
       return;
@@ -58,4 +62,4 @@ public class RemoveTrivialCalcRule extends RelOptRule {
   }
 }
 
-// End RemoveTrivialCalcRule.java
+// End CalcRemoveRule.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/rules/CoerceInputsRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/CoerceInputsRule.java b/core/src/main/java/org/apache/calcite/rel/rules/CoerceInputsRule.java
index 5393654..5f489b9 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/CoerceInputsRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/CoerceInputsRule.java
@@ -14,15 +14,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
+package org.apache.calcite.rel.rules;
+
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.type.RelDataType;
 
 import java.util.ArrayList;
 import java.util.List;
 
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.*;
-
 /**
  * CoerceInputsRule precasts inputs to a particular type. This can be used to
  * assist operator implementations which impose requirements on their input

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/rules/EmptyPruneRules.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/EmptyPruneRules.java b/core/src/main/java/org/apache/calcite/rel/rules/EmptyPruneRules.java
index a73fa83..9f3cd67 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/EmptyPruneRules.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/EmptyPruneRules.java
@@ -14,28 +14,43 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
+package org.apache.calcite.rel.rules;
 
-import java.util.*;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.SingleRel;
+import org.apache.calcite.rel.core.Aggregate;
+import org.apache.calcite.rel.core.Empty;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.logical.LogicalUnion;
+import org.apache.calcite.rex.RexLiteral;
 
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.rex.RexLiteral;
+import java.util.ArrayList;
+import java.util.List;
 
-import static org.eigenbase.relopt.RelOptRule.*;
+import static org.apache.calcite.plan.RelOptRule.any;
+import static org.apache.calcite.plan.RelOptRule.none;
+import static org.apache.calcite.plan.RelOptRule.operand;
+import static org.apache.calcite.plan.RelOptRule.some;
+import static org.apache.calcite.plan.RelOptRule.unordered;
 
 /**
  * Collection of rules which remove sections of a query plan known never to
  * produce any rows.
  *
- * @see EmptyRel
+ * @see org.apache.calcite.rel.core.Empty
  */
-public abstract class RemoveEmptyRules {
+public abstract class EmptyPruneRules {
   //~ Static fields/initializers ---------------------------------------------
 
   /**
    * Rule that removes empty children of a
-   * {@link UnionRel}.
+   * {@link org.apache.calcite.rel.logical.LogicalUnion}.
    *
    * <p>Examples:
    *
@@ -47,20 +62,20 @@ public abstract class RemoveEmptyRules {
    */
   public static final RelOptRule UNION_INSTANCE =
       new RelOptRule(
-          operand(UnionRel.class,
-              unordered(operand(EmptyRel.class, none()))),
+          operand(LogicalUnion.class,
+              unordered(operand(Empty.class, none()))),
           "Union") {
         public void onMatch(RelOptRuleCall call) {
-          UnionRel union = call.rel(0);
+          LogicalUnion union = call.rel(0);
           final List<RelNode> childRels = call.getChildRels(union);
           final List<RelNode> newChildRels = new ArrayList<RelNode>();
           for (RelNode childRel : childRels) {
-            if (!(childRel instanceof EmptyRel)) {
+            if (!(childRel instanceof Empty)) {
               newChildRels.add(childRel);
             }
           }
           assert newChildRels.size() < childRels.size()
-              : "planner promised us at least one EmptyRel child";
+              : "planner promised us at least one Empty child";
           RelNode newRel;
           switch (newChildRels.size()) {
           case 0:
@@ -75,7 +90,7 @@ public abstract class RemoveEmptyRules {
             break;
           default:
             newRel =
-                new UnionRel(
+                new LogicalUnion(
                     union.getCluster(),
                     newChildRels,
                     union.all);
@@ -86,7 +101,7 @@ public abstract class RemoveEmptyRules {
       };
 
   /**
-   * Rule that converts a {@link ProjectRel}
+   * Rule that converts a {@link org.apache.calcite.rel.logical.LogicalProject}
    * to empty if its child is empty.
    *
    * <p>Examples:
@@ -96,10 +111,10 @@ public abstract class RemoveEmptyRules {
    * </ul>
    */
   public static final RelOptRule PROJECT_INSTANCE =
-      new RemoveEmptySingleRule(ProjectRelBase.class, "PruneEmptyProject");
+      new RemoveEmptySingleRule(Project.class, "PruneEmptyProject");
 
   /**
-   * Rule that converts a {@link FilterRel}
+   * Rule that converts a {@link org.apache.calcite.rel.logical.LogicalFilter}
    * to empty if its child is empty.
    *
    * <p>Examples:
@@ -109,10 +124,10 @@ public abstract class RemoveEmptyRules {
    * </ul>
    */
   public static final RelOptRule FILTER_INSTANCE =
-      new RemoveEmptySingleRule(FilterRelBase.class, "PruneEmptyFilter");
+      new RemoveEmptySingleRule(Filter.class, "PruneEmptyFilter");
 
   /**
-   * Rule that converts a {@link SortRel}
+   * Rule that converts a {@link org.apache.calcite.rel.core.Sort}
    * to empty if its child is empty.
    *
    * <p>Examples:
@@ -122,10 +137,10 @@ public abstract class RemoveEmptyRules {
    * </ul>
    */
   public static final RelOptRule SORT_INSTANCE =
-      new RemoveEmptySingleRule(SortRel.class, "PruneEmptySort");
+      new RemoveEmptySingleRule(Sort.class, "PruneEmptySort");
 
   /**
-   * Rule that converts a {@link SortRel}
+   * Rule that converts a {@link org.apache.calcite.rel.core.Sort}
    * to empty if it has {@code LIMIT 0}.
    *
    * <p>Examples:
@@ -136,10 +151,9 @@ public abstract class RemoveEmptyRules {
    */
   public static final RelOptRule SORT_FETCH_ZERO_INSTANCE =
       new RelOptRule(
-          operand(SortRel.class, any()), "PruneSortLimit0") {
-        @Override
-        public void onMatch(RelOptRuleCall call) {
-          SortRel sort = call.rel(0);
+          operand(Sort.class, any()), "PruneSortLimit0") {
+        @Override public void onMatch(RelOptRuleCall call) {
+          Sort sort = call.rel(0);
           if (sort.fetch != null
               && RexLiteral.intValue(sort.fetch) == 0) {
             call.transformTo(empty(sort));
@@ -148,7 +162,7 @@ public abstract class RemoveEmptyRules {
       };
 
   /**
-   * Rule that converts an {@link AggregateRelBase}
+   * Rule that converts an {@link org.apache.calcite.rel.core.Aggregate}
    * to empty if its child is empty.
    *
    * <p>Examples:
@@ -158,10 +172,10 @@ public abstract class RemoveEmptyRules {
    * </ul>
    */
   public static final RelOptRule AGGREGATE_INSTANCE =
-      new RemoveEmptySingleRule(AggregateRelBase.class, "PruneEmptyAggregate");
+      new RemoveEmptySingleRule(Aggregate.class, "PruneEmptyAggregate");
 
   /**
-   * Rule that converts a {@link JoinRelBase}
+   * Rule that converts a {@link org.apache.calcite.rel.core.Join}
    * to empty if its left child is empty.
    *
    * <p>Examples:
@@ -172,14 +186,13 @@ public abstract class RemoveEmptyRules {
    */
   public static final RelOptRule JOIN_LEFT_INSTANCE =
       new RelOptRule(
-          operand(JoinRelBase.class,
+          operand(Join.class,
               some(
-                  operand(EmptyRel.class, none()),
+                  operand(Empty.class, none()),
                   operand(RelNode.class, any()))),
               "PruneEmptyJoin(left)") {
-        @Override
-        public void onMatch(RelOptRuleCall call) {
-          JoinRelBase join = call.rel(0);
+        @Override public void onMatch(RelOptRuleCall call) {
+          Join join = call.rel(0);
           if (join.getJoinType().generatesNullsOnLeft()) {
             // "select * from emp right join dept" is not necessarily empty if
             // emp is empty
@@ -190,7 +203,7 @@ public abstract class RemoveEmptyRules {
       };
 
   /**
-   * Rule that converts a {@link JoinRelBase}
+   * Rule that converts a {@link org.apache.calcite.rel.core.Join}
    * to empty if its right child is empty.
    *
    * <p>Examples:
@@ -201,14 +214,13 @@ public abstract class RemoveEmptyRules {
    */
   public static final RelOptRule JOIN_RIGHT_INSTANCE =
       new RelOptRule(
-          operand(JoinRelBase.class,
+          operand(Join.class,
               some(
                   operand(RelNode.class, any()),
-                  operand(EmptyRel.class, none()))),
+                  operand(Empty.class, none()))),
               "PruneEmptyJoin(right)") {
-        @Override
-        public void onMatch(RelOptRuleCall call) {
-          JoinRelBase join = call.rel(0);
+        @Override public void onMatch(RelOptRuleCall call) {
+          Join join = call.rel(0);
           if (join.getJoinType().generatesNullsOnRight()) {
             // "select * from emp left join dept" is not necessarily empty if
             // dept is empty
@@ -218,9 +230,10 @@ public abstract class RemoveEmptyRules {
         }
       };
 
-  /** Creates an {@link EmptyRel} to replace {@code node}. */
-  private static EmptyRel empty(RelNode node) {
-    return new EmptyRel(node.getCluster(), node.getRowType());
+  /** Creates an {@link org.apache.calcite.rel.core.Empty} to replace
+   * {@code node}. */
+  private static Empty empty(RelNode node) {
+    return new Empty(node.getCluster(), node.getRowType());
   }
 
   /** Planner rule that converts a single-rel (e.g. project, sort, aggregate or
@@ -230,7 +243,7 @@ public abstract class RemoveEmptyRules {
         String description) {
       super(
           operand(clazz,
-              operand(EmptyRel.class, none())),
+              operand(Empty.class, none())),
           description);
     }
 
@@ -241,4 +254,4 @@ public abstract class RemoveEmptyRules {
   }
 }
 
-// End RemoveEmptyRules.java
+// End EmptyPruneRules.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/rules/EquiJoin.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/EquiJoin.java b/core/src/main/java/org/apache/calcite/rel/rules/EquiJoin.java
index 884bdbd..d017f1e 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/EquiJoin.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/EquiJoin.java
@@ -14,30 +14,30 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
+package org.apache.calcite.rel.rules;
 
-import java.util.Set;
-
-import org.eigenbase.rel.JoinInfo;
-import org.eigenbase.rel.JoinRelBase;
-import org.eigenbase.rel.JoinRelType;
-import org.eigenbase.rel.RelNode;
-import org.eigenbase.relopt.RelOptCluster;
-import org.eigenbase.relopt.RelTraitSet;
-import org.eigenbase.rex.RexNode;
-import org.eigenbase.util.ImmutableIntList;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.JoinInfo;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.util.ImmutableIntList;
 
 import com.google.common.base.Preconditions;
 
+import java.util.Set;
+
 /**
  * Base class for any join whose condition is based on column equality.
  */
-public abstract class EquiJoinRel extends JoinRelBase {
+public abstract class EquiJoin extends Join {
   public final ImmutableIntList leftKeys;
   public final ImmutableIntList rightKeys;
 
-  /** Creates an EquiJoinRel. */
-  public EquiJoinRel(RelOptCluster cluster, RelTraitSet traits, RelNode left,
+  /** Creates an EquiJoin. */
+  public EquiJoin(RelOptCluster cluster, RelTraitSet traits, RelNode left,
       RelNode right, RexNode condition, ImmutableIntList leftKeys,
       ImmutableIntList rightKeys, JoinRelType joinType,
       Set<String> variablesStopped) {
@@ -54,10 +54,9 @@ public abstract class EquiJoinRel extends JoinRelBase {
     return rightKeys;
   }
 
-  @Override
-  public JoinInfo analyzeCondition() {
+  @Override public JoinInfo analyzeCondition() {
     return JoinInfo.of(leftKeys, rightKeys);
   }
 }
 
-// End EquiJoinRel.java
+// End EquiJoin.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/rules/FilterAggregateTransposeRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/FilterAggregateTransposeRule.java b/core/src/main/java/org/apache/calcite/rel/rules/FilterAggregateTransposeRule.java
index a0f3b67..c3a7e25 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/FilterAggregateTransposeRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/FilterAggregateTransposeRule.java
@@ -14,26 +14,31 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
+package org.apache.calcite.rel.rules;
 
-import java.util.BitSet;
-import java.util.List;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.RelDataTypeField;
-import org.eigenbase.rex.*;
-
-import net.hydromatic.optiq.util.BitSets;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Aggregate;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.core.RelFactories;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.util.BitSets;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 
+import java.util.BitSet;
+import java.util.List;
+
 /**
- * Planner rule that pushes a {@link FilterRelBase}
- * past a {@link AggregateRelBase}.
+ * Planner rule that pushes a {@link org.apache.calcite.rel.core.Filter}
+ * past a {@link org.apache.calcite.rel.core.Aggregate}.
  *
- * @see org.eigenbase.rel.rules.AggregateFilterTransposeRule
+ * @see org.apache.calcite.rel.rules.AggregateFilterTransposeRule
  */
 public class FilterAggregateTransposeRule extends RelOptRule {
 
@@ -42,9 +47,9 @@ public class FilterAggregateTransposeRule extends RelOptRule {
    *
    * <p>It matches any kind of agg. or filter */
   public static final FilterAggregateTransposeRule INSTANCE =
-      new FilterAggregateTransposeRule(FilterRelBase.class,
+      new FilterAggregateTransposeRule(Filter.class,
           RelFactories.DEFAULT_FILTER_FACTORY,
-          AggregateRelBase.class);
+          Aggregate.class);
 
   private final RelFactories.FilterFactory filterFactory;
 
@@ -57,9 +62,9 @@ public class FilterAggregateTransposeRule extends RelOptRule {
    * matched in the rule. Similarly {@code aggregateFactory}.</p>
    */
   public FilterAggregateTransposeRule(
-      Class<? extends FilterRelBase> filterClass,
+      Class<? extends Filter> filterClass,
       RelFactories.FilterFactory filterFactory,
-      Class<? extends AggregateRelBase> aggregateClass) {
+      Class<? extends Aggregate> aggregateClass) {
     super(
         operand(filterClass,
             operand(aggregateClass, any())));
@@ -70,8 +75,8 @@ public class FilterAggregateTransposeRule extends RelOptRule {
 
   // implement RelOptRule
   public void onMatch(RelOptRuleCall call) {
-    final FilterRelBase filterRel = call.rel(0);
-    final AggregateRelBase aggRel = call.rel(1);
+    final Filter filterRel = call.rel(0);
+    final Aggregate aggRel = call.rel(1);
 
     final List<RexNode> conditions =
         RelOptUtil.conjunctions(filterRel.getCondition());

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/rules/FilterCalcMergeRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/FilterCalcMergeRule.java b/core/src/main/java/org/apache/calcite/rel/rules/FilterCalcMergeRule.java
index a881b62..5257ee6 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/FilterCalcMergeRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/FilterCalcMergeRule.java
@@ -14,41 +14,49 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
+package org.apache.calcite.rel.rules;
 
-import java.util.*;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.logical.LogicalCalc;
+import org.apache.calcite.rel.logical.LogicalFilter;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexProgramBuilder;
 
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.rex.*;
+import java.util.Collections;
 
 /**
- * Planner rule which merges a {@link FilterRel} and a {@link CalcRel}. The
- * result is a {@link CalcRel} whose filter condition is the logical AND of the
- * two.
+ * Planner rule that merges a
+ * {@link org.apache.calcite.rel.logical.LogicalFilter} and a
+ * {@link org.apache.calcite.rel.logical.LogicalCalc}. The
+ * result is a {@link org.apache.calcite.rel.logical.LogicalCalc}
+ * whose filter condition is the logical AND of the two.
  *
- * @see MergeFilterOntoCalcRule
+ * @see FilterMergeRule
  */
-public class MergeFilterOntoCalcRule extends RelOptRule {
+public class FilterCalcMergeRule extends RelOptRule {
   //~ Static fields/initializers ---------------------------------------------
 
-  public static final MergeFilterOntoCalcRule INSTANCE =
-      new MergeFilterOntoCalcRule();
+  public static final FilterCalcMergeRule INSTANCE =
+      new FilterCalcMergeRule();
 
   //~ Constructors -----------------------------------------------------------
 
-  private MergeFilterOntoCalcRule() {
+  private FilterCalcMergeRule() {
     super(
         operand(
-            FilterRelBase.class,
-            operand(CalcRel.class, any())));
+            Filter.class,
+            operand(LogicalCalc.class, any())));
   }
 
   //~ Methods ----------------------------------------------------------------
 
   public void onMatch(RelOptRuleCall call) {
-    final FilterRel filter = call.rel(0);
-    final CalcRel calc = call.rel(1);
+    final LogicalFilter filter = call.rel(0);
+    final LogicalCalc calc = call.rel(1);
 
     // Don't merge a filter onto a calc which contains windowed aggregates.
     // That would effectively be pushing a multiset down through a filter.
@@ -74,11 +82,11 @@ public class MergeFilterOntoCalcRule extends RelOptRule {
             topProgram,
             bottomProgram,
             rexBuilder);
-    final CalcRel newCalc =
-        new CalcRel(
+    final LogicalCalc newCalc =
+        new LogicalCalc(
             calc.getCluster(),
             calc.getTraitSet(),
-            calc.getChild(),
+            calc.getInput(),
             filter.getRowType(),
             mergedProgram,
             Collections.<RelCollation>emptyList());
@@ -86,4 +94,4 @@ public class MergeFilterOntoCalcRule extends RelOptRule {
   }
 }
 
-// End MergeFilterOntoCalcRule.java
+// End FilterCalcMergeRule.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/rules/FilterJoinRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/FilterJoinRule.java b/core/src/main/java/org/apache/calcite/rel/rules/FilterJoinRule.java
index b169568..9ea4c40 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/FilterJoinRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/FilterJoinRule.java
@@ -14,34 +14,45 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
-
-import java.util.*;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.rex.*;
+package org.apache.calcite.rel.rules;
+
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptRuleOperand;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.core.RelFactories;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexUtil;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
 
+import java.util.ArrayList;
+import java.util.List;
+
 /**
- * PushFilterPastJoinRule implements the rule for pushing filters above and
+ * Planner rule that pushes filters above and
  * within a join node into the join node and/or its children nodes.
  */
-public abstract class PushFilterPastJoinRule extends RelOptRule {
-  public static final PushFilterPastJoinRule FILTER_ON_JOIN =
-      new PushFilterIntoJoinRule(true);
+public abstract class FilterJoinRule extends RelOptRule {
+  public static final FilterJoinRule FILTER_ON_JOIN =
+      new FilterIntoJoinRule(true);
 
   /** Dumber version of {@link #FILTER_ON_JOIN}. Not intended for production
    * use, but keeps some tests working for which {@code FILTER_ON_JOIN} is too
    * smart. */
-  public static final PushFilterPastJoinRule DUMB_FILTER_ON_JOIN =
-      new PushFilterIntoJoinRule(false);
+  public static final FilterJoinRule DUMB_FILTER_ON_JOIN =
+      new FilterIntoJoinRule(false);
 
-  public static final PushFilterPastJoinRule JOIN =
-      new PushDownJoinConditionRule(RelFactories.DEFAULT_FILTER_FACTORY,
+  public static final FilterJoinRule JOIN =
+      new JoinConditionPushRule(RelFactories.DEFAULT_FILTER_FACTORY,
           RelFactories.DEFAULT_PROJECT_FACTORY);
 
   /** Whether to try to strengthen join-type. */
@@ -54,10 +65,10 @@ public abstract class PushFilterPastJoinRule extends RelOptRule {
   //~ Constructors -----------------------------------------------------------
 
   /**
-   * Creates a PushFilterPastJoinRule with an explicit root operand and
+   * Creates a FilterJoinRule with an explicit root operand and
    * factories.
    */
-  protected PushFilterPastJoinRule(RelOptRuleOperand operand, String id,
+  protected FilterJoinRule(RelOptRuleOperand operand, String id,
       boolean smart, RelFactories.FilterFactory filterFactory,
       RelFactories.ProjectFactory projectFactory) {
     super(operand, "PushFilterRule: " + id);
@@ -68,8 +79,8 @@ public abstract class PushFilterPastJoinRule extends RelOptRule {
 
   //~ Methods ----------------------------------------------------------------
 
-  protected void perform(RelOptRuleCall call, FilterRelBase filter,
-      JoinRelBase join) {
+  protected void perform(RelOptRuleCall call, Filter filter,
+      Join join) {
     final List<RexNode> joinFilters =
         RelOptUtil.conjunctions(join.getCondition());
     final List<RexNode> origJoinFilters = ImmutableList.copyOf(joinFilters);
@@ -114,7 +125,7 @@ public abstract class PushFilterPastJoinRule extends RelOptRule {
         join,
         aboveFilters,
         joinType,
-        !(join instanceof EquiJoinRel),
+        !(join instanceof EquiJoin),
         !joinType.generatesNullsOnLeft(),
         !joinType.generatesNullsOnRight(),
         joinFilters,
@@ -205,7 +216,7 @@ public abstract class PushFilterPastJoinRule extends RelOptRule {
     newJoinRel = RelOptUtil.createCastRel(newJoinRel, join.getRowType(),
         false, projectFactory);
 
-    // create a FilterRel on top of the join if needed
+    // create a LogicalFilter on top of the join if needed
     RelNode newRel =
         RelOptUtil.createFilter(newJoinRel, aboveFilters, filterFactory);
 
@@ -226,12 +237,13 @@ public abstract class PushFilterPastJoinRule extends RelOptRule {
    * @param joinFilters Filters in join condition
    * @param join Join
    *
-   * @deprecated Use {@link #validateJoinFilters(java.util.List, java.util.List, org.eigenbase.rel.JoinRelBase, org.eigenbase.rel.JoinRelType)};
+   * @deprecated Use
+   * {@link #validateJoinFilters(java.util.List, java.util.List, org.apache.calcite.rel.core.Join, org.apache.calcite.rel.core.JoinRelType)};
    * very short-term; will be removed before
-   * {@link org.eigenbase.util.Bug#upgrade(String) calcite-0.9.2}.
+   * {@link org.apache.calcite.util.Bug#upgrade(String) calcite-0.9.2}.
    */
   protected void validateJoinFilters(List<RexNode> aboveFilters,
-      List<RexNode> joinFilters, JoinRelBase join) {
+      List<RexNode> joinFilters, Join join) {
     validateJoinFilters(aboveFilters, joinFilters, join, join.getJoinType());
   }
 
@@ -252,52 +264,49 @@ public abstract class PushFilterPastJoinRule extends RelOptRule {
    * outer join simplification.
    */
   protected void validateJoinFilters(List<RexNode> aboveFilters,
-      List<RexNode> joinFilters, JoinRelBase join, JoinRelType joinType) {
+      List<RexNode> joinFilters, Join join, JoinRelType joinType) {
     return;
   }
 
   /** Rule that pushes parts of the join condition to its inputs. */
-  public static class PushDownJoinConditionRule
-      extends PushFilterPastJoinRule {
-    public PushDownJoinConditionRule(RelFactories.FilterFactory filterFactory,
+  public static class JoinConditionPushRule extends FilterJoinRule {
+    public JoinConditionPushRule(RelFactories.FilterFactory filterFactory,
         RelFactories.ProjectFactory projectFactory) {
-      super(RelOptRule.operand(JoinRelBase.class, RelOptRule.any()),
-          "PushFilterPastJoinRule:no-filter",
+      super(RelOptRule.operand(Join.class, RelOptRule.any()),
+          "FilterJoinRule:no-filter",
           true, filterFactory, projectFactory);
     }
 
-    @Override
-    public void onMatch(RelOptRuleCall call) {
-      JoinRelBase join = call.rel(0);
+    @Override public void onMatch(RelOptRuleCall call) {
+      Join join = call.rel(0);
       perform(call, null, join);
     }
   }
 
   /** Rule that tries to push filter expressions into a join
    * condition and into the inputs of the join. */
-  public static class PushFilterIntoJoinRule extends PushFilterPastJoinRule {
-    public PushFilterIntoJoinRule(boolean smart) {
+  public static class FilterIntoJoinRule extends FilterJoinRule {
+    public FilterIntoJoinRule(boolean smart) {
       this(smart, RelFactories.DEFAULT_FILTER_FACTORY,
           RelFactories.DEFAULT_PROJECT_FACTORY);
     }
 
-    public PushFilterIntoJoinRule(boolean smart,
+    public FilterIntoJoinRule(boolean smart,
         RelFactories.FilterFactory filterFactory,
         RelFactories.ProjectFactory projectFactory) {
       super(
-          RelOptRule.operand(FilterRelBase.class,
-              RelOptRule.operand(JoinRelBase.class, RelOptRule.any())),
-          "PushFilterPastJoinRule:filter",
+          RelOptRule.operand(Filter.class,
+              RelOptRule.operand(Join.class, RelOptRule.any())),
+          "FilterJoinRule:filter",
           smart, filterFactory, projectFactory);
     }
 
-    @Override
-    public void onMatch(RelOptRuleCall call) {
-      FilterRelBase filter = call.rel(0);
-      JoinRelBase join = call.rel(1);
+    @Override public void onMatch(RelOptRuleCall call) {
+      Filter filter = call.rel(0);
+      Join join = call.rel(1);
       perform(call, filter, join);
     }
   }
 }
 
-// End PushFilterPastJoinRule.java
+// End FilterJoinRule.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/rules/FilterMergeRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/FilterMergeRule.java b/core/src/main/java/org/apache/calcite/rel/rules/FilterMergeRule.java
index b059bc0..418e5c8 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/FilterMergeRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/FilterMergeRule.java
@@ -14,30 +14,37 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
+package org.apache.calcite.rel.rules;
 
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.rex.*;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.core.RelFactories;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexProgramBuilder;
+import org.apache.calcite.rex.RexUtil;
 
 /**
- * MergeFilterRule implements the rule for combining two {@link FilterRel}s
+ * Planner rule that combines two
+ * {@link org.apache.calcite.rel.logical.LogicalFilter}s.
  */
-public class MergeFilterRule extends RelOptRule {
-  public static final MergeFilterRule INSTANCE =
-      new MergeFilterRule(RelFactories.DEFAULT_FILTER_FACTORY);
+public class FilterMergeRule extends RelOptRule {
+  public static final FilterMergeRule INSTANCE =
+      new FilterMergeRule(RelFactories.DEFAULT_FILTER_FACTORY);
 
   private final RelFactories.FilterFactory filterFactory;
 
   //~ Constructors -----------------------------------------------------------
 
   /**
-   * Creates a MergeFilterRule.
+   * Creates a FilterMergeRule.
    */
-  public MergeFilterRule(RelFactories.FilterFactory filterFactory) {
+  public FilterMergeRule(RelFactories.FilterFactory filterFactory) {
     super(
-        operand(FilterRelBase.class,
-            operand(FilterRelBase.class, any())));
+        operand(Filter.class,
+            operand(Filter.class, any())));
     this.filterFactory = filterFactory;
   }
 
@@ -45,12 +52,12 @@ public class MergeFilterRule extends RelOptRule {
 
   // implement RelOptRule
   public void onMatch(RelOptRuleCall call) {
-    final FilterRelBase topFilter = call.rel(0);
-    final FilterRelBase bottomFilter = call.rel(1);
+    final Filter topFilter = call.rel(0);
+    final Filter bottomFilter = call.rel(1);
 
     // use RexPrograms to merge the two FilterRels into a single program
-    // so we can convert the two FilterRel conditions to directly
-    // reference the bottom FilterRel's child
+    // so we can convert the two LogicalFilter conditions to directly
+    // reference the bottom LogicalFilter's child
     RexBuilder rexBuilder = topFilter.getCluster().getRexBuilder();
     RexProgram bottomProgram = createProgram(bottomFilter);
     RexProgram topProgram = createProgram(topFilter);
@@ -65,21 +72,21 @@ public class MergeFilterRule extends RelOptRule {
         mergedProgram.expandLocalRef(
             mergedProgram.getCondition());
 
-    FilterRelBase newFilterRel =
-        (FilterRelBase) filterFactory.createFilter(
-            bottomFilter.getChild(),
+    Filter newFilterRel =
+        (Filter) filterFactory.createFilter(
+            bottomFilter.getInput(),
             RexUtil.flatten(rexBuilder, newCondition));
 
     call.transformTo(newFilterRel);
   }
 
   /**
-   * Creates a RexProgram corresponding to a FilterRel
+   * Creates a RexProgram corresponding to a LogicalFilter
    *
-   * @param filterRel the FilterRel
+   * @param filterRel the LogicalFilter
    * @return created RexProgram
    */
-  private RexProgram createProgram(FilterRelBase filterRel) {
+  private RexProgram createProgram(Filter filterRel) {
     RexProgramBuilder programBuilder =
         new RexProgramBuilder(
             filterRel.getRowType(),
@@ -90,4 +97,4 @@ public class MergeFilterRule extends RelOptRule {
   }
 }
 
-// End MergeFilterRule.java
+// End FilterMergeRule.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a0ba73cd/core/src/main/java/org/apache/calcite/rel/rules/FilterMultiJoinMergeRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/FilterMultiJoinMergeRule.java b/core/src/main/java/org/apache/calcite/rel/rules/FilterMultiJoinMergeRule.java
index 7c73043..fe446d1 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/FilterMultiJoinMergeRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/FilterMultiJoinMergeRule.java
@@ -14,53 +14,56 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.eigenbase.rel.rules;
+package org.apache.calcite.rel.rules;
 
-import org.eigenbase.rel.*;
-import org.eigenbase.relopt.*;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.rel.logical.LogicalFilter;
 
 /**
- * PushFilterIntoMultiJoinRule implements the rule for pushing a {@link
- * FilterRel} into a {@link MultiJoinRel}.
+ * Planner rule that merges a
+ * {@link org.apache.calcite.rel.logical.LogicalFilter}
+ * into a {@link MultiJoin},
+ * creating a richer {@code MultiJoin}.
+ *
+ * @see org.apache.calcite.rel.rules.ProjectMultiJoinMergeRule
  */
-public class PushFilterIntoMultiJoinRule extends RelOptRule {
-  public static final PushFilterIntoMultiJoinRule INSTANCE =
-      new PushFilterIntoMultiJoinRule();
+public class FilterMultiJoinMergeRule extends RelOptRule {
+  public static final FilterMultiJoinMergeRule INSTANCE =
+      new FilterMultiJoinMergeRule();
 
   //~ Constructors -----------------------------------------------------------
 
   /**
-   * Creates a PushFilterIntoMultiJoinRule.
+   * Creates a FilterMultiJoinMergeRule.
    */
-  private PushFilterIntoMultiJoinRule() {
+  private FilterMultiJoinMergeRule() {
     super(
-        operand(
-            FilterRel.class,
-            operand(MultiJoinRel.class, any())));
+        operand(LogicalFilter.class,
+            operand(MultiJoin.class, any())));
   }
 
   //~ Methods ----------------------------------------------------------------
 
-  // implement RelOptRule
   public void onMatch(RelOptRuleCall call) {
-    FilterRel filterRel = call.rel(0);
-    MultiJoinRel multiJoinRel = call.rel(1);
+    LogicalFilter filter = call.rel(0);
+    MultiJoin multiJoin = call.rel(1);
 
-    MultiJoinRel newMultiJoinRel =
-        new MultiJoinRel(
-            multiJoinRel.getCluster(),
-            multiJoinRel.getInputs(),
-            multiJoinRel.getJoinFilter(),
-            multiJoinRel.getRowType(),
-            multiJoinRel.isFullOuterJoin(),
-            multiJoinRel.getOuterJoinConditions(),
-            multiJoinRel.getJoinTypes(),
-            multiJoinRel.getProjFields(),
-            multiJoinRel.getJoinFieldRefCountsMap(),
-            filterRel.getCondition());
+    MultiJoin newMultiJoin =
+        new MultiJoin(
+            multiJoin.getCluster(),
+            multiJoin.getInputs(),
+            multiJoin.getJoinFilter(),
+            multiJoin.getRowType(),
+            multiJoin.isFullOuterJoin(),
+            multiJoin.getOuterJoinConditions(),
+            multiJoin.getJoinTypes(),
+            multiJoin.getProjFields(),
+            multiJoin.getJoinFieldRefCountsMap(),
+            filter.getCondition());
 
-    call.transformTo(newMultiJoinRel);
+    call.transformTo(newMultiJoin);
   }
 }
 
-// End PushFilterIntoMultiJoinRule.java
+// End FilterMultiJoinMergeRule.java