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/09/03 21:48:04 UTC

[1/3] git commit: HOWTO: modify Optiq web site

Repository: incubator-optiq
Updated Branches:
  refs/heads/master df9b682d0 -> 771ec5449


HOWTO: modify Optiq web site


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

Branch: refs/heads/master
Commit: 973f2f8870060adc0acd23e8d08fe3d2b4021ad0
Parents: df9b682
Author: Julian Hyde <jh...@apache.org>
Authored: Wed Sep 3 11:14:30 2014 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Wed Sep 3 11:14:30 2014 -0700

----------------------------------------------------------------------
 doc/HOWTO.md | 29 +++++++++++++++++++++++++++++
 1 file changed, 29 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/973f2f88/doc/HOWTO.md
----------------------------------------------------------------------
diff --git a/doc/HOWTO.md b/doc/HOWTO.md
index 2d0e8f5..c703c50 100644
--- a/doc/HOWTO.md
+++ b/doc/HOWTO.md
@@ -558,3 +558,32 @@ out to mirrors, and other tasks.
   with a change comment
   "Resolved in release X.Y.Z-incubating (YYYY-MM-DD)"
   (fill in release number and date appropriately).
+
+## Publishing the web site (for Optiq committers)
+
+Get the code:
+
+```bash
+$ svn co https://svn.apache.org/repos/asf/incubator/optiq/site optiq-site
+```
+
+(Note: `https:`, not `http:`.)
+
+Build the site:
+
+```bash
+$ cd optiq-site
+$ ./build.sh
+```
+
+It will prompt you to install jekyll, redcarpet and pygments, if you
+do not have them installed. It will also check out the git source code
+repo, so that it can generate javadoc.
+
+Check in:
+
+```bash
+svn ci -m"Commit message" file...
+```
+
+The site will automatically be deployed as http://optiq.incubator.apache.org.


[2/3] git commit: [OPTIQ-396] Change return type of JoinFactory.createJoin(); add SemiJoinFactory

Posted by jh...@apache.org.
[OPTIQ-396] Change return type of JoinFactory.createJoin(); add SemiJoinFactory

Close apache/incubator-optiq#11


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

Branch: refs/heads/master
Commit: e34f7f0f894f3d6d503210f941236bae03c441f2
Parents: 973f2f8
Author: Ashutosh Chauhan <ha...@apache.org>
Authored: Tue Sep 2 16:27:21 2014 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Wed Sep 3 11:41:38 2014 -0700

----------------------------------------------------------------------
 .../java/org/eigenbase/rel/RelFactories.java    | 78 ++++++++++++--------
 .../rel/rules/OptimizeBushyJoinRule.java        |  2 +-
 .../org/eigenbase/sql2rel/RelFieldTrimmer.java  | 32 ++++----
 3 files changed, 66 insertions(+), 46 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/e34f7f0f/core/src/main/java/org/eigenbase/rel/RelFactories.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/rel/RelFactories.java b/core/src/main/java/org/eigenbase/rel/RelFactories.java
index 0bb7788..8a722a2 100644
--- a/core/src/main/java/org/eigenbase/rel/RelFactories.java
+++ b/core/src/main/java/org/eigenbase/rel/RelFactories.java
@@ -46,6 +46,9 @@ public class RelFactories {
 
   public static final JoinFactory DEFAULT_JOIN_FACTORY = new JoinFactoryImpl();
 
+  public static final SemiJoinFactory DEFAULT_SEMI_JOIN_FACTORY =
+      new SemiJoinFactoryImpl();
+
   public static final SortFactory DEFAULT_SORT_FACTORY =
     new SortFactoryImpl();
 
@@ -63,16 +66,13 @@ public class RelFactories {
    * for this rule's calling convention.
    */
   public interface ProjectFactory {
-    /**
-     * Can create a {@link org.eigenbase.rel.ProjectRel} of the appropriate type
-     * for this rule's calling convention.
-     */
+    /** Creates a project. */
     RelNode createProject(RelNode child, List<RexNode> childExprs,
         List<String> fieldNames);
   }
 
   /**
-   * Implementation of {@link ProjectFactory} that returns vanilla
+   * Implementation of {@link ProjectFactory} that returns a vanilla
    * {@link ProjectRel}.
    */
   private static class ProjectFactoryImpl implements ProjectFactory {
@@ -87,13 +87,14 @@ public class RelFactories {
    * for this rule's calling convention.
    */
   public interface SortFactory {
+    /** Creates a sort. */
     RelNode createSort(RelTraitSet traits, RelNode child,
         RelCollation collation, RexNode offset, RexNode fetch);
   }
 
   /**
    * Implementation of {@link org.eigenbase.rel.RelFactories.SortFactory} that
-   * returns vanilla {@link SortRel}.
+   * returns a vanilla {@link SortRel}.
    */
   private static class SortFactoryImpl implements SortFactory {
     public RelNode createSort(RelTraitSet traits, RelNode child,
@@ -109,6 +110,7 @@ public class RelFactories {
    * for this rule's calling convention.
    */
   public interface SetOpFactory {
+    /** Creates a set operation. */
     RelNode createSetOp(SqlKind kind, List<RelNode> inputs, boolean all);
   }
 
@@ -139,17 +141,17 @@ public class RelFactories {
    * for this rule's calling convention.
    */
   public interface AggregateFactory {
-    RelNode createAggrRelNode(RelNode child, BitSet groupSet,
-      List<AggregateCall> aggCalls);
+    /** Creates an aggregate. */
+    RelNode createAggregate(RelNode child, BitSet groupSet,
+        List<AggregateCall> aggCalls);
   }
 
   /**
-   * Implementation of {@link org.eigenbase.rel.RelFactories.AggregateFactory} that
-   * returns vanilla {@link AggregateRel}.
+   * Implementation of {@link org.eigenbase.rel.RelFactories.AggregateFactory}
+   * that returns a vanilla {@link AggregateRel}.
    */
   private static class AggregateFactoryImpl implements AggregateFactory {
-
-    public RelNode createAggrRelNode(RelNode child, BitSet groupSet,
+    public RelNode createAggregate(RelNode child, BitSet groupSet,
         List<AggregateCall> aggCalls) {
       return new AggregateRel(child.getCluster(), child, groupSet, aggCalls);
     }
@@ -160,16 +162,13 @@ public class RelFactories {
    * for this rule's calling convention.
    */
   public interface FilterFactory {
-    /**
-     * Can create a {@link org.eigenbase.rel.FilterRel} of the appropriate type
-     * for this rule's calling convention.
-     */
+    /** Creates a filter. */
     RelNode createFilter(RelNode child, RexNode condition);
   }
 
   /**
    * Implementation of {@link org.eigenbase.rel.RelFactories.FilterFactory} that
-   * returns vanilla {@link FilterRel}.
+   * returns a vanilla {@link FilterRel}.
    */
   private static class FilterFactoryImpl implements FilterFactory {
     public RelNode createFilter(RelNode child, RexNode condition) {
@@ -178,8 +177,9 @@ public class RelFactories {
   }
 
   /**
-   * Can create a {@link org.eigenbase.rel.JoinRelBase} of the appropriate type
-   * for this rule's calling convention.
+   * 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}.
    */
   public interface JoinFactory {
     /**
@@ -198,28 +198,46 @@ public class RelFactories {
     RelNode createJoin(RelNode left, RelNode right, RexNode condition,
         JoinRelType joinType, Set<String> variablesStopped,
         boolean semiJoinDone);
-
-    SemiJoinRel createSemiJoinRel(RelTraitSet traitSet, RelNode left,
-      RelNode right, RexNode condition);
   }
 
   /**
-   * Implementation of {@link JoinFactory} that returns vanilla
+   * Implementation of {@link JoinFactory} that returns a vanilla
    * {@link JoinRel}.
    */
   private static class JoinFactoryImpl implements JoinFactory {
-    public RelNode createJoin(RelNode left, RelNode right, RexNode condition,
-        JoinRelType joinType, Set<String> variablesStopped,
-        boolean semiJoinDone) {
+    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,
           variablesStopped, semiJoinDone, ImmutableList.<RelDataTypeField>of());
     }
+  }
 
-    public SemiJoinRel createSemiJoinRel(RelTraitSet traitSet, RelNode left,
-        RelNode right, RexNode condition) {
+  /**
+   * Can create a semi-join of the appropriate type for a rule's calling
+   * convention.
+   */
+  public interface SemiJoinFactory {
+    /**
+     * Creates a semi-join.
+     *
+     * @param left             Left input
+     * @param right            Right input
+     * @param condition        Join condition
+     */
+    RelNode createSemiJoin(RelNode left, RelNode right, RexNode condition);
+  }
+
+  /**
+   * Implementation of {@link SemiJoinFactory} that returns a vanilla
+   * {@link SemiJoinRel}.
+   */
+  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(), traitSet, left, right,
+      return new SemiJoinRel(left.getCluster(), left.getTraitSet(), left, right,
         condition, joinInfo.leftKeys, joinInfo.rightKeys);
     }
   }
@@ -246,12 +264,10 @@ public class RelFactories {
     final RexBuilder rexBuilder = child.getCluster().getRexBuilder();
     return factory.createProject(child,
         new AbstractList<RexNode>() {
-          @Override
           public int size() {
             return posList.size();
           }
 
-          @Override
           public RexNode get(int index) {
             final int pos = posList.get(index);
             return rexBuilder.makeInputRef(child, pos);

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/e34f7f0f/core/src/main/java/org/eigenbase/rel/rules/OptimizeBushyJoinRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/rel/rules/OptimizeBushyJoinRule.java b/core/src/main/java/org/eigenbase/rel/rules/OptimizeBushyJoinRule.java
index 22bf75c..cfa8f01 100644
--- a/core/src/main/java/org/eigenbase/rel/rules/OptimizeBushyJoinRule.java
+++ b/core/src/main/java/org/eigenbase/rel/rules/OptimizeBushyJoinRule.java
@@ -242,7 +242,7 @@ public class OptimizeBushyJoinRule extends RelOptRule {
             RexUtil.composeConjunction(rexBuilder, joinVertex.conditions,
                 false);
         relNodes.add(
-            Pair.of(
+            Pair.of((RelNode)
                 joinFactory.createJoin(left, right, condition.accept(shuttle),
                     JoinRelType.INNER, ImmutableSet.<String>of(), false),
                 mapping));

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/e34f7f0f/core/src/main/java/org/eigenbase/sql2rel/RelFieldTrimmer.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/sql2rel/RelFieldTrimmer.java b/core/src/main/java/org/eigenbase/sql2rel/RelFieldTrimmer.java
index cf740b7..d1b7c95 100644
--- a/core/src/main/java/org/eigenbase/sql2rel/RelFieldTrimmer.java
+++ b/core/src/main/java/org/eigenbase/sql2rel/RelFieldTrimmer.java
@@ -68,6 +68,7 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
   private final RelFactories.ProjectFactory projectFactory;
   private final RelFactories.FilterFactory filterFactory;
   private final RelFactories.JoinFactory joinFactory;
+  private final RelFactories.SemiJoinFactory semiJoinFactory;
   private final RelFactories.SortFactory sortFactory;
   private final RelFactories.AggregateFactory aggregateFactory;
   private final RelFactories.SetOpFactory setOpFactory;
@@ -84,6 +85,7 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
         RelFactories.DEFAULT_PROJECT_FACTORY,
         RelFactories.DEFAULT_FILTER_FACTORY,
         RelFactories.DEFAULT_JOIN_FACTORY,
+        RelFactories.DEFAULT_SEMI_JOIN_FACTORY,
         RelFactories.DEFAULT_SORT_FACTORY,
         RelFactories.DEFAULT_AGGREGATE_FACTORY,
         RelFactories.DEFAULT_SET_OP_FACTORY);
@@ -95,12 +97,13 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
    * @param validator Validator
    */
   public RelFieldTrimmer(SqlValidator validator,
-    RelFactories.ProjectFactory projectFactory,
-    RelFactories.FilterFactory filterFactory,
-    RelFactories.JoinFactory joinFactory,
-    RelFactories.SortFactory sortFactory,
-    RelFactories.AggregateFactory aggregateFactory,
-    RelFactories.SetOpFactory setOpFactory) {
+      RelFactories.ProjectFactory projectFactory,
+      RelFactories.FilterFactory filterFactory,
+      RelFactories.JoinFactory joinFactory,
+      RelFactories.SemiJoinFactory semiJoinFactory,
+      RelFactories.SortFactory sortFactory,
+      RelFactories.AggregateFactory aggregateFactory,
+      RelFactories.SetOpFactory setOpFactory) {
     Util.discard(validator); // may be useful one day
     this.trimFieldsDispatcher =
         ReflectUtil.createMethodDispatcher(
@@ -113,6 +116,7 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
     this.projectFactory = Preconditions.checkNotNull(projectFactory);
     this.filterFactory = Preconditions.checkNotNull(filterFactory);
     this.joinFactory = Preconditions.checkNotNull(joinFactory);
+    this.semiJoinFactory = Preconditions.checkNotNull(semiJoinFactory);
     this.sortFactory = Preconditions.checkNotNull(sortFactory);
     this.aggregateFactory = Preconditions.checkNotNull(aggregateFactory);
     this.setOpFactory = Preconditions.checkNotNull(setOpFactory);
@@ -596,10 +600,10 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
     RexNode newConditionExpr =
         conditionExpr.accept(shuttle);
 
-    final JoinRelBase newJoin;
+    final RelNode newJoin;
     if (join instanceof SemiJoinRel) {
-      newJoin = joinFactory.createSemiJoinRel(join.getTraitSet(),
-          newInputs.get(0), newInputs.get(1), newConditionExpr);
+      newJoin = semiJoinFactory.createSemiJoin(newInputs.get(0),
+          newInputs.get(1), newConditionExpr);
       // For SemiJoins only map fields from the left-side
       Mapping inputMapping = inputMappings.get(0);
       mapping = Mappings.create(MappingType.INVERSE_SURJECTION,
@@ -614,9 +618,9 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
         mapping.set(pair.source + offset, pair.target + newOffset);
       }
     } else {
-      newJoin = (JoinRelBase) joinFactory.createJoin(
-        newInputs.get(0), newInputs.get(1), newConditionExpr,
-        join.getJoinType(), join.getVariablesStopped(), join.isSemiJoinDone());
+      newJoin = joinFactory.createJoin(newInputs.get(0), newInputs.get(1),
+          newConditionExpr, join.getJoinType(), join.getVariablesStopped(),
+          join.isSemiJoinDone());
     }
 
     return new TrimResult(newJoin, mapping);
@@ -797,8 +801,8 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
       ++j;
     }
 
-    RelNode newAggregate = aggregateFactory.createAggrRelNode(newInput,
-      newGroupSet, newAggCallList);
+    RelNode newAggregate = aggregateFactory.createAggregate(newInput,
+        newGroupSet, newAggCallList);
 
     assert newAggregate.getClass() == aggregate.getClass();
 


[3/3] git commit: [OPTIQ-398] Move CalcRel.createProject methods to RelOptUtil

Posted by jh...@apache.org.
[OPTIQ-398] Move CalcRel.createProject methods to RelOptUtil


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

Branch: refs/heads/master
Commit: 771ec5449a0e8cea1f2a55626b9850b64a5a4937
Parents: e34f7f0
Author: Julian Hyde <jh...@apache.org>
Authored: Wed Sep 3 12:20:44 2014 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Wed Sep 3 12:20:44 2014 -0700

----------------------------------------------------------------------
 .../main/java/org/eigenbase/rel/CalcRel.java    | 336 +----------------
 .../main/java/org/eigenbase/rel/ProjectRel.java |   2 +-
 .../java/org/eigenbase/rel/ProjectRelBase.java  |   2 +-
 .../java/org/eigenbase/rel/RelFactories.java    |  46 +--
 .../main/java/org/eigenbase/rel/WindowRel.java  |   2 +-
 .../rel/rules/ExtractJoinFilterRule.java        |   2 +-
 .../rel/rules/NestedLoopsJoinRule.java          |   2 +-
 .../rel/rules/OptimizeBushyJoinRule.java        |   2 +-
 .../PullConstantsThroughAggregatesRule.java     |   4 +-
 .../rel/rules/PushJoinThroughJoinRule.java      |   4 +-
 .../rel/rules/PushProjectIntoMultiJoinRule.java |   2 +-
 .../org/eigenbase/rel/rules/PushProjector.java  |   6 +-
 .../rel/rules/PushSemiJoinPastFilterRule.java   |   2 +-
 .../rel/rules/PushSemiJoinPastProjectRule.java  |   2 +-
 .../rel/rules/ReduceAggregatesRule.java         |   4 +-
 .../rel/rules/ReduceExpressionsRule.java        |   2 +-
 .../rel/rules/RemoveDistinctAggregateRule.java  |   5 +-
 .../rel/rules/RemoveIsNotDistinctFromRule.java  |   2 +-
 .../rel/rules/RemoveTrivialProjectRule.java     |   2 +-
 .../org/eigenbase/rel/rules/SwapJoinRule.java   |   2 +-
 .../rel/rules/WindowedAggSplitterRule.java      |   2 +-
 .../eigenbase/relopt/RelOptMaterialization.java |   6 +-
 .../java/org/eigenbase/relopt/RelOptUtil.java   | 359 +++++++++++++++++--
 .../eigenbase/relopt/SubstitutionVisitor.java   |   9 +-
 .../main/java/org/eigenbase/rex/RexProgram.java |   7 +-
 .../org/eigenbase/sql2rel/RelDecorrelator.java  |  22 +-
 .../org/eigenbase/sql2rel/RelFieldTrimmer.java  |   4 +-
 .../sql2rel/RelStructuredTypeFlattener.java     |   6 +-
 .../eigenbase/sql2rel/SqlToRelConverter.java    |  26 +-
 .../relopt/volcano/VolcanoPlannerTest.java      |   8 +-
 30 files changed, 417 insertions(+), 463 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/771ec544/core/src/main/java/org/eigenbase/rel/CalcRel.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/rel/CalcRel.java b/core/src/main/java/org/eigenbase/rel/CalcRel.java
index 1812786..ba0b650 100644
--- a/core/src/main/java/org/eigenbase/rel/CalcRel.java
+++ b/core/src/main/java/org/eigenbase/rel/CalcRel.java
@@ -22,17 +22,14 @@ import org.eigenbase.rel.rules.*;
 import org.eigenbase.relopt.*;
 import org.eigenbase.reltype.*;
 import org.eigenbase.rex.*;
-import org.eigenbase.sql.validate.SqlValidatorUtil;
-import org.eigenbase.util.*;
-import org.eigenbase.util.mapping.*;
 
 /**
  * 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 latter stages
- * of optimization, by merging consecutive {@link ProjectRel} and {@link
- * FilterRel} nodes together.
+ * <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.
  *
  * <p>The following rules relate to <code>CalcRel</code>:</p>
  *
@@ -40,18 +37,17 @@ import org.eigenbase.util.mapping.*;
  * <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 MergeProjectOntoCalcRule} merges this with a
+ *     {@link ProjectRel}</li>
  * <li>{@link MergeCalcRule} merges two CalcRels</li>
  * </ul>
  */
 public final class CalcRel extends CalcRelBase {
   //~ Static fields/initializers ---------------------------------------------
 
-  public static final boolean DEPRECATE_PROJECT_AND_FILTER = false;
-
   //~ Constructors -----------------------------------------------------------
 
+  /** Creates a CalcRel. */
   public CalcRel(
       RelOptCluster cluster,
       RelTraitSet traits,
@@ -64,185 +60,13 @@ public final class CalcRel extends CalcRelBase {
 
   //~ Methods ----------------------------------------------------------------
 
-  @Override
-  public CalcRelBase copy(
-      RelTraitSet traitSet, RelNode child,
+  @Override public CalcRelBase copy(RelTraitSet traitSet, RelNode child,
       RexProgram program, List<RelCollation> collationList) {
-    return new CalcRel(
-        getCluster(), traitSet, child, program.getOutputRowType(), program,
-        collationList);
-  }
-
-  /**
-   * Creates a relational expression which projects an array of expressions.
-   *
-   * @param child         input relational expression
-   * @param exprList      list of expressions for the input columns
-   * @param fieldNameList aliases of the expressions, or null to generate
-   */
-  public static RelNode createProject(
-      RelNode child,
-      List<RexNode> exprList,
-      List<String> fieldNameList) {
-    return createProject(child, exprList, fieldNameList, false);
-  }
-
-  /**
-   * Creates a relational expression which projects an array of expressions.
-   *
-   * @param child       input relational expression
-   * @param projectList list of (expression, name) pairs
-   * @param optimize    Whether to optimize
-   */
-  public static RelNode createProject(
-      RelNode child,
-      List<Pair<RexNode, String>> projectList,
-      boolean optimize) {
-    return createProject(
-        child, Pair.left(projectList), Pair.right(projectList), optimize);
-  }
-
-  /**
-   * Creates a relational expression that projects the given fields of the
-   * input.
-   *
-   * <p>Optimizes if the fields are the identity projection.</p>
-   *
-   * @param child   Input relational expression
-   * @param posList Source of each projected field
-   * @return Relational expression that projects given fields
-   */
-  public static RelNode createProject(final RelNode child,
-      final List<Integer> posList) {
-    return RelFactories.createProject(RelFactories.DEFAULT_PROJECT_FACTORY,
-        child, posList);
-  }
-
-  /**
-   * Creates a relational expression which projects an array of expressions,
-   * and optionally optimizes.
-   *
-   * <p>The result may not be a {@link ProjectRel}. If the projection is
-   * trivial, <code>child</code> is returned directly; and future versions may
-   * return other formulations of expressions, such as {@link CalcRel}.
-   *
-   * @param child      input relational expression
-   * @param exprs      list of expressions for the input columns
-   * @param fieldNames aliases of the expressions, or null to generate
-   * @param optimize   Whether to return <code>child</code> unchanged if the
-   *                   projections are trivial.
-   */
-  public static RelNode createProject(
-      RelNode child,
-      List<RexNode> exprs,
-      List<String> fieldNames,
-      boolean optimize) {
-    final RelOptCluster cluster = child.getCluster();
-    final RexProgram program =
-        RexProgram.create(
-            child.getRowType(), exprs, null, fieldNames,
-            cluster.getRexBuilder());
-    final List<RelCollation> collationList =
-        program.getCollations(child.getCollationList());
-    if (DEPRECATE_PROJECT_AND_FILTER) {
-      return new CalcRel(
-          cluster,
-          child.getTraitSet(),
-          child,
-          program.getOutputRowType(),
-          program,
-          collationList);
-    } else {
-      final RelDataType rowType =
-          RexUtil.createStructType(
-              cluster.getTypeFactory(),
-              exprs,
-              fieldNames == null
-                  ? null
-                  : SqlValidatorUtil.uniquify(
-                      fieldNames, SqlValidatorUtil.F_SUGGESTER));
-      if (optimize
-          && RemoveTrivialProjectRule.isIdentity(exprs, rowType,
-              child.getRowType())) {
-        return child;
-      }
-      return
-          new ProjectRel(
-              cluster,
-              cluster.traitSetOf(
-                  collationList.isEmpty()
-                      ? RelCollationImpl.EMPTY
-                      : collationList.get(0)),
-              child,
-              exprs,
-              rowType,
-              ProjectRelBase.Flags.BOXED);
-    }
+    return new CalcRel(getCluster(), traitSet, child,
+        program.getOutputRowType(), program, collationList);
   }
 
-  /**
-   * Creates a relational expression which filters according to a given
-   * condition, returning the same fields as its input.
-   *
-   * @param child     Child relational expression
-   * @param condition Condition
-   * @return Relational expression
-   */
-  public static RelNode createFilter(
-      RelNode child,
-      RexNode condition) {
-    if (DEPRECATE_PROJECT_AND_FILTER) {
-      final RelOptCluster cluster = child.getCluster();
-      RexProgramBuilder builder =
-          new RexProgramBuilder(
-              child.getRowType(),
-              cluster.getRexBuilder());
-      builder.addIdentity();
-      builder.addCondition(condition);
-      final RexProgram program = builder.getProgram();
-      return new CalcRel(
-          cluster,
-          child.getTraitSet(),
-          child,
-          program.getOutputRowType(),
-          program,
-          Collections.<RelCollation>emptyList());
-    } else {
-      return new FilterRel(
-          child.getCluster(),
-          child,
-          condition);
-    }
-  }
-
-  /**
-   * Returns a relational expression which has the same fields as the
-   * underlying expression, but the fields have different names.
-   *
-   * @param rel        Relational expression
-   * @param fieldNames Field names
-   * @return Renamed relational expression
-   */
-  public static RelNode createRename(
-      RelNode rel,
-      List<String> fieldNames) {
-    final List<RelDataTypeField> fields = rel.getRowType().getFieldList();
-    assert fieldNames.size() == fields.size();
-    final List<Pair<RexNode, String>> refs =
-        new AbstractList<Pair<RexNode, String>>() {
-          public int size() {
-            return fields.size();
-          }
-
-          public Pair<RexNode, String> get(int index) {
-            return RexInputRef.of2(index, fields);
-          }
-        };
-    return createProject(rel, refs, true);
-  }
-
-  @Override
-  public void collectVariablesUsed(Set<String> variableSet) {
+  @Override public void collectVariablesUsed(Set<String> variableSet) {
     final RelOptUtil.VariableUsedVisitor vuv =
         new RelOptUtil.VariableUsedVisitor();
     for (RexNode expr : program.getExprList()) {
@@ -250,144 +74,6 @@ public final class CalcRel extends CalcRelBase {
     }
     variableSet.addAll(vuv.variables);
   }
-
-  /**
-   * Creates a relational expression which permutes the output fields of a
-   * relational expression according to a permutation.
-   *
-   * <p>Optimizations:</p>
-   *
-   * <ul>
-   * <li>If the relational expression is a {@link CalcRel} or {@link
-   * ProjectRel} which 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>
-   * </ul>
-   *
-   * <p>If a permutation is combined with its inverse, these optimizations
-   * would combine to remove them both.
-   *
-   * @param rel         Relational expression
-   * @param permutation Permutation to apply to fields
-   * @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 permutes its input fields
-   */
-  public static RelNode permute(
-      RelNode rel,
-      Permutation permutation,
-      List<String> fieldNames) {
-    if (permutation.isIdentity()) {
-      return rel;
-    }
-    if (rel instanceof CalcRel) {
-      CalcRel calcRel = (CalcRel) rel;
-      Permutation permutation1 = calcRel.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 (permutation1 != null) {
-        Permutation permutation2 = permutation.product(permutation1);
-        return permute(rel, permutation2, null);
-      }
-    }
-    final List<RelDataType> outputTypeList = new ArrayList<RelDataType>();
-    final List<String> outputNameList = new ArrayList<String>();
-    final List<RexNode> exprList = new ArrayList<RexNode>();
-    final List<RexLocalRef> projectRefList = new ArrayList<RexLocalRef>();
-    final List<RelDataTypeField> fields = rel.getRowType().getFieldList();
-    for (int i = 0; i < permutation.getTargetCount(); i++) {
-      int target = permutation.getTarget(i);
-      final RelDataTypeField targetField = fields.get(target);
-      outputTypeList.add(targetField.getType());
-      outputNameList.add(
-          ((fieldNames == null)
-              || (fieldNames.size() <= i)
-              || (fieldNames.get(i) == null)) ? targetField.getName()
-              : fieldNames.get(i));
-      exprList.add(
-          rel.getCluster().getRexBuilder().makeInputRef(
-              fields.get(i).getType(),
-              i));
-      final int source = permutation.getSource(i);
-      projectRefList.add(
-          new RexLocalRef(
-              source,
-              fields.get(source).getType()));
-    }
-    final RexProgram program =
-        new RexProgram(
-            rel.getRowType(),
-            exprList,
-            projectRefList,
-            null,
-            rel.getCluster().getTypeFactory().createStructType(
-                outputTypeList,
-                outputNameList));
-    return new CalcRel(
-        rel.getCluster(),
-        rel.getTraitSet(),
-        rel,
-        program.getOutputRowType(),
-        program,
-        Collections.<RelCollation>emptyList());
-  }
-
-  /**
-   * Creates a relational expression which projects the output fields of a
-   * relational expression according to a partial mapping.
-   *
-   * <p>A partial mapping is weaker than a permutation: every target has one
-   * source, but a source may have 0, 1 or more than one targets. Usually the
-   * result will have fewer fields than the source, unless some source fields
-   * are projected multiple times.
-   *
-   * <p>This method could optimize the result as {@link #permute} does, but
-   * 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 MappingType#isMandatorySource()} and
-   *                   {@link MappingType#isSingleSource()}, as does
-   *                   {@link 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
-   */
-  public static RelNode projectMapping(
-      RelNode rel,
-      Mapping mapping,
-      List<String> fieldNames,
-      RelFactories.ProjectFactory projectFactory) {
-    assert mapping.getMappingType().isSingleSource();
-    assert mapping.getMappingType().isMandatorySource();
-    if (mapping.isIdentity()) {
-      return rel;
-    }
-    final List<String> outputNameList = new ArrayList<String>();
-    final List<RexNode> exprList = new ArrayList<RexNode>();
-    final List<RelDataTypeField> fields = rel.getRowType().getFieldList();
-    final RexBuilder rexBuilder = rel.getCluster().getRexBuilder();
-    for (int i = 0; i < fields.size(); i++) {
-      exprList.add(rexBuilder.makeInputRef(rel, i));
-    }
-    for (int i = 0; i < mapping.getTargetCount(); i++) {
-      int source = mapping.getSource(i);
-      final RelDataTypeField sourceField = fields.get(source);
-      outputNameList.add(
-          ((fieldNames == null)
-              || (fieldNames.size() <= i)
-              || (fieldNames.get(i) == null)) ? sourceField.getName()
-              : fieldNames.get(i));
-    }
-    return projectFactory.createProject(rel, exprList, outputNameList);
-  }
 }
 
 // End CalcRel.java

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/771ec544/core/src/main/java/org/eigenbase/rel/ProjectRel.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/rel/ProjectRel.java b/core/src/main/java/org/eigenbase/rel/ProjectRel.java
index feb31a8..d7a5bf1 100644
--- a/core/src/main/java/org/eigenbase/rel/ProjectRel.java
+++ b/core/src/main/java/org/eigenbase/rel/ProjectRel.java
@@ -75,7 +75,7 @@ public final class ProjectRel extends ProjectRelBase {
       RelOptCluster cluster,
       RelTraitSet traitSet,
       RelNode child,
-      List<RexNode> exps,
+      List<? extends RexNode> exps,
       RelDataType rowType,
       int flags) {
     super(cluster, traitSet, child, exps, rowType, flags);

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/771ec544/core/src/main/java/org/eigenbase/rel/ProjectRelBase.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/rel/ProjectRelBase.java b/core/src/main/java/org/eigenbase/rel/ProjectRelBase.java
index 583e2d8..83786b4 100644
--- a/core/src/main/java/org/eigenbase/rel/ProjectRelBase.java
+++ b/core/src/main/java/org/eigenbase/rel/ProjectRelBase.java
@@ -66,7 +66,7 @@ public abstract class ProjectRelBase extends SingleRel {
       RelOptCluster cluster,
       RelTraitSet traits,
       RelNode child,
-      List<RexNode> exps,
+      List<? extends RexNode> exps,
       RelDataType rowType,
       int flags) {
     super(cluster, traits, child);

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/771ec544/core/src/main/java/org/eigenbase/rel/RelFactories.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/rel/RelFactories.java b/core/src/main/java/org/eigenbase/rel/RelFactories.java
index 8a722a2..b5d9634 100644
--- a/core/src/main/java/org/eigenbase/rel/RelFactories.java
+++ b/core/src/main/java/org/eigenbase/rel/RelFactories.java
@@ -17,19 +17,17 @@
 
 package org.eigenbase.rel;
 
-import java.util.AbstractList;
 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.RexBuilder;
 import org.eigenbase.rex.RexNode;
 import org.eigenbase.sql.SqlKind;
-import org.eigenbase.util.mapping.Mappings;
 
 import com.google.common.collect.ImmutableList;
 
@@ -67,7 +65,7 @@ public class RelFactories {
    */
   public interface ProjectFactory {
     /** Creates a project. */
-    RelNode createProject(RelNode child, List<RexNode> childExprs,
+    RelNode createProject(RelNode child, List<? extends RexNode> childExprs,
         List<String> fieldNames);
   }
 
@@ -76,9 +74,9 @@ public class RelFactories {
    * {@link ProjectRel}.
    */
   private static class ProjectFactoryImpl implements ProjectFactory {
-    public RelNode createProject(RelNode child, List<RexNode> childExprs,
-        List<String> fieldNames) {
-      return CalcRel.createProject(child, childExprs, fieldNames);
+    public RelNode createProject(RelNode child,
+        List<? extends RexNode> childExprs, List<String> fieldNames) {
+      return RelOptUtil.createProject(child, childExprs, fieldNames);
     }
   }
 
@@ -241,40 +239,6 @@ public class RelFactories {
         condition, joinInfo.leftKeys, joinInfo.rightKeys);
     }
   }
-
-  /**
-   * Creates a relational expression that projects the given fields of the
-   * input.
-   *
-   * <p>Optimizes if the fields are the identity projection.
-   *
-   * @param factory
-   *          ProjectFactory
-   * @param child
-   *          Input relational expression
-   * @param posList
-   *          Source of each projected field
-   * @return Relational expression that projects given fields
-   */
-  public static RelNode createProject(final ProjectFactory factory,
-      final RelNode child, final List<Integer> posList) {
-    if (Mappings.isIdentity(posList, child.getRowType().getFieldCount())) {
-      return child;
-    }
-    final RexBuilder rexBuilder = child.getCluster().getRexBuilder();
-    return factory.createProject(child,
-        new AbstractList<RexNode>() {
-          public int size() {
-            return posList.size();
-          }
-
-          public RexNode get(int index) {
-            final int pos = posList.get(index);
-            return rexBuilder.makeInputRef(child, pos);
-          }
-        },
-        null);
-  }
 }
 
 // End RelFactories.java

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/771ec544/core/src/main/java/org/eigenbase/rel/WindowRel.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/rel/WindowRel.java b/core/src/main/java/org/eigenbase/rel/WindowRel.java
index ed373e0..d278a38 100644
--- a/core/src/main/java/org/eigenbase/rel/WindowRel.java
+++ b/core/src/main/java/org/eigenbase/rel/WindowRel.java
@@ -239,7 +239,7 @@ public final class WindowRel extends WindowRelBase {
             cluster, traitSet, child, constants, intermediateRowType,
             windowList);
 
-    return CalcRel.createProject(
+    return RelOptUtil.createProject(
         window,
         toInputRefs(program.getProjectList()),
         outRowType.getFieldNames());

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/771ec544/core/src/main/java/org/eigenbase/rel/rules/ExtractJoinFilterRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/rel/rules/ExtractJoinFilterRule.java b/core/src/main/java/org/eigenbase/rel/rules/ExtractJoinFilterRule.java
index d4c0a81..39563d3 100644
--- a/core/src/main/java/org/eigenbase/rel/rules/ExtractJoinFilterRule.java
+++ b/core/src/main/java/org/eigenbase/rel/rules/ExtractJoinFilterRule.java
@@ -77,7 +77,7 @@ public final class ExtractJoinFilterRule extends RelOptRule {
             join.isSemiJoinDone());
 
     RelNode filterRel =
-        CalcRel.createFilter(
+        RelOptUtil.createFilter(
             cartesianJoinRel,
             join.getCondition());
 

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/771ec544/core/src/main/java/org/eigenbase/rel/rules/NestedLoopsJoinRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/rel/rules/NestedLoopsJoinRule.java b/core/src/main/java/org/eigenbase/rel/rules/NestedLoopsJoinRule.java
index a450d5f..f0fed59 100644
--- a/core/src/main/java/org/eigenbase/rel/rules/NestedLoopsJoinRule.java
+++ b/core/src/main/java/org/eigenbase/rel/rules/NestedLoopsJoinRule.java
@@ -110,7 +110,7 @@ public class NestedLoopsJoinRule extends RelOptRule {
                         dynInIdStr)));
       }
       right =
-          CalcRel.createFilter(
+          RelOptUtil.createFilter(
               right,
               condition);
     }

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/771ec544/core/src/main/java/org/eigenbase/rel/rules/OptimizeBushyJoinRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/rel/rules/OptimizeBushyJoinRule.java b/core/src/main/java/org/eigenbase/rel/rules/OptimizeBushyJoinRule.java
index cfa8f01..007e49e 100644
--- a/core/src/main/java/org/eigenbase/rel/rules/OptimizeBushyJoinRule.java
+++ b/core/src/main/java/org/eigenbase/rel/rules/OptimizeBushyJoinRule.java
@@ -254,7 +254,7 @@ public class OptimizeBushyJoinRule extends RelOptRule {
 
     final Pair<RelNode, Mappings.TargetMapping> top = Util.last(relNodes);
     final RelNode project =
-        RelFactories.createProject(projectFactory, top.left,
+        RelOptUtil.createProject(projectFactory, top.left,
             Mappings.asList(top.right));
 
     call.transformTo(project);

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/771ec544/core/src/main/java/org/eigenbase/rel/rules/PullConstantsThroughAggregatesRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/rel/rules/PullConstantsThroughAggregatesRule.java b/core/src/main/java/org/eigenbase/rel/rules/PullConstantsThroughAggregatesRule.java
index 26d1be8..6d741f7 100644
--- a/core/src/main/java/org/eigenbase/rel/rules/PullConstantsThroughAggregatesRule.java
+++ b/core/src/main/java/org/eigenbase/rel/rules/PullConstantsThroughAggregatesRule.java
@@ -200,7 +200,7 @@ public class PullConstantsThroughAggregatesRule extends RelOptRule {
       projects.add(Pair.of(expr, field.getName()));
     }
     final RelNode inverseProject =
-        CalcRel.createProject(newAggregate, projects, false);
+        RelOptUtil.createProject(newAggregate, projects, false);
 
     call.transformTo(inverseProject);
   }
@@ -234,7 +234,7 @@ public class PullConstantsThroughAggregatesRule extends RelOptRule {
               (RexNode) rexBuilder.makeInputRef(child, source),
               childRowType.getFieldList().get(source).getName()));
     }
-    return CalcRel.createProject(child, projects, false);
+    return RelOptUtil.createProject(child, projects, false);
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/771ec544/core/src/main/java/org/eigenbase/rel/rules/PushJoinThroughJoinRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/rel/rules/PushJoinThroughJoinRule.java b/core/src/main/java/org/eigenbase/rel/rules/PushJoinThroughJoinRule.java
index d26486f..76b7f2e 100644
--- a/core/src/main/java/org/eigenbase/rel/rules/PushJoinThroughJoinRule.java
+++ b/core/src/main/java/org/eigenbase/rel/rules/PushJoinThroughJoinRule.java
@@ -189,7 +189,7 @@ public class PushJoinThroughJoinRule extends RelOptRule {
             relB, topJoin.getJoinType(), topJoin.isSemiJoinDone());
 
     assert !Mappings.isIdentity(topMapping);
-    final RelNode newProject = RelFactories.createProject(projectFactory,
+    final RelNode newProject = RelOptUtil.createProject(projectFactory,
         newTopJoin, Mappings.asList(topMapping));
 
     call.transformTo(newProject);
@@ -296,7 +296,7 @@ public class PushJoinThroughJoinRule extends RelOptRule {
         topJoin.copy(topJoin.getTraitSet(), newTopCondition, newBottomJoin,
             relA, topJoin.getJoinType(), topJoin.isSemiJoinDone());
 
-    final RelNode newProject = RelFactories.createProject(projectFactory,
+    final RelNode newProject = RelOptUtil.createProject(projectFactory,
         newTopJoin, Mappings.asList(topMapping));
 
     call.transformTo(newProject);

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/771ec544/core/src/main/java/org/eigenbase/rel/rules/PushProjectIntoMultiJoinRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/rel/rules/PushProjectIntoMultiJoinRule.java b/core/src/main/java/org/eigenbase/rel/rules/PushProjectIntoMultiJoinRule.java
index 8eeb64a..fd300c9 100644
--- a/core/src/main/java/org/eigenbase/rel/rules/PushProjectIntoMultiJoinRule.java
+++ b/core/src/main/java/org/eigenbase/rel/rules/PushProjectIntoMultiJoinRule.java
@@ -64,7 +64,7 @@ public class PushProjectIntoMultiJoinRule extends RelOptRule {
     MultiJoinRel newMultiJoin =
         RelOptUtil.projectMultiJoin(multiJoin, project);
     ProjectRel newProject =
-        (ProjectRel) CalcRel.createProject(
+        (ProjectRel) RelOptUtil.createProject(
             newMultiJoin,
             project.getProjects(),
             project.getRowType().getFieldNames());

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/771ec544/core/src/main/java/org/eigenbase/rel/rules/PushProjector.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/rel/rules/PushProjector.java b/core/src/main/java/org/eigenbase/rel/rules/PushProjector.java
index b35e252..b51e3b8 100644
--- a/core/src/main/java/org/eigenbase/rel/rules/PushProjector.java
+++ b/core/src/main/java/org/eigenbase/rel/rules/PushProjector.java
@@ -296,7 +296,7 @@ public class PushProjector {
               origFilter,
               newProject.getRowType().getFieldList(),
               adjustments);
-      projChild = CalcRel.createFilter(newProject, newFilter);
+      projChild = RelOptUtil.createFilter(newProject, newFilter);
     } else {
       projChild = newProject;
     }
@@ -464,7 +464,7 @@ public class PushProjector {
               ((RexCall) projExpr).getOperator().getName()));
     }
 
-    return (ProjectRel) CalcRel.createProject(
+    return (ProjectRel) RelOptUtil.createProject(
         projChild,
         Pair.left(newProjects),
         Pair.right(newProjects));
@@ -552,7 +552,7 @@ public class PushProjector {
                     field.e.getType(), field.i), field.e.getName()));
       }
     }
-    return (ProjectRel) CalcRel.createProject(
+    return (ProjectRel) RelOptUtil.createProject(
         projChild,
         Pair.left(projects),
         Pair.right(projects),

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/771ec544/core/src/main/java/org/eigenbase/rel/rules/PushSemiJoinPastFilterRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/rel/rules/PushSemiJoinPastFilterRule.java b/core/src/main/java/org/eigenbase/rel/rules/PushSemiJoinPastFilterRule.java
index 6067c8e..53d2132 100644
--- a/core/src/main/java/org/eigenbase/rel/rules/PushSemiJoinPastFilterRule.java
+++ b/core/src/main/java/org/eigenbase/rel/rules/PushSemiJoinPastFilterRule.java
@@ -58,7 +58,7 @@ public class PushSemiJoinPastFilterRule extends RelOptRule {
             semiJoin.getRightKeys());
 
     RelNode newFilter =
-        CalcRel.createFilter(
+        RelOptUtil.createFilter(
             newSemiJoin,
             filter.getCondition());
 

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/771ec544/core/src/main/java/org/eigenbase/rel/rules/PushSemiJoinPastProjectRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/rel/rules/PushSemiJoinPastProjectRule.java b/core/src/main/java/org/eigenbase/rel/rules/PushSemiJoinPastProjectRule.java
index 7154037..d6efb37 100644
--- a/core/src/main/java/org/eigenbase/rel/rules/PushSemiJoinPastProjectRule.java
+++ b/core/src/main/java/org/eigenbase/rel/rules/PushSemiJoinPastProjectRule.java
@@ -84,7 +84,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 =
-        CalcRel.createProject(
+        RelOptUtil.createProject(
             newSemiJoin,
             projExprs,
             project.getRowType().getFieldNames());

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/771ec544/core/src/main/java/org/eigenbase/rel/rules/ReduceAggregatesRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/rel/rules/ReduceAggregatesRule.java b/core/src/main/java/org/eigenbase/rel/rules/ReduceAggregatesRule.java
index bca974a..11f9c15 100644
--- a/core/src/main/java/org/eigenbase/rel/rules/ReduceAggregatesRule.java
+++ b/core/src/main/java/org/eigenbase/rel/rules/ReduceAggregatesRule.java
@@ -131,7 +131,7 @@ public class ReduceAggregatesRule extends RelOptRule {
         inputExprs.size() - input.getRowType().getFieldCount();
     if (extraArgCount > 0) {
       input =
-          CalcRel.createProject(
+          RelOptUtil.createProject(
               input,
               inputExprs,
               CompositeList.of(
@@ -145,7 +145,7 @@ public class ReduceAggregatesRule extends RelOptRule {
             oldAggRel, input, newCalls);
 
     RelNode projectRel =
-        CalcRel.createProject(
+        RelOptUtil.createProject(
             newAggRel,
             projList,
             oldAggRel.getRowType().getFieldNames());

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/771ec544/core/src/main/java/org/eigenbase/rel/rules/ReduceExpressionsRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/rel/rules/ReduceExpressionsRule.java b/core/src/main/java/org/eigenbase/rel/rules/ReduceExpressionsRule.java
index 0027b38..7025a47 100644
--- a/core/src/main/java/org/eigenbase/rel/rules/ReduceExpressionsRule.java
+++ b/core/src/main/java/org/eigenbase/rel/rules/ReduceExpressionsRule.java
@@ -87,7 +87,7 @@ public abstract class ReduceExpressionsRule extends RelOptRule {
                     filter.getRowType()));
           } else if (reduced) {
             call.transformTo(
-                CalcRel.createFilter(
+                RelOptUtil.createFilter(
                     filter.getChild(),
                     expList.get(0)));
           } else {

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/771ec544/core/src/main/java/org/eigenbase/rel/rules/RemoveDistinctAggregateRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/rel/rules/RemoveDistinctAggregateRule.java b/core/src/main/java/org/eigenbase/rel/rules/RemoveDistinctAggregateRule.java
index f81eec3..560f13d 100644
--- a/core/src/main/java/org/eigenbase/rel/rules/RemoveDistinctAggregateRule.java
+++ b/core/src/main/java/org/eigenbase/rel/rules/RemoveDistinctAggregateRule.java
@@ -137,8 +137,7 @@ public final class RemoveDistinctAggregateRule extends RelOptRule {
       rel = doRewrite(aggregate, rel, argList, refs);
     }
 
-    //noinspection unchecked
-    rel = CalcRel.createProject(rel, (List) refs, fieldNames);
+    rel = RelOptUtil.createProject(rel, refs, fieldNames);
 
     call.transformTo(rel);
   }
@@ -464,7 +463,7 @@ public final class RemoveDistinctAggregateRule extends RelOptRule {
       projects.add(RexInputRef.of2(arg, childFields));
     }
     final RelNode project =
-        CalcRel.createProject(child, projects, false);
+        RelOptUtil.createProject(child, projects, false);
 
     // Get the distinct values of the GROUP BY fields and the arguments
     // to the agg functions.

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/771ec544/core/src/main/java/org/eigenbase/rel/rules/RemoveIsNotDistinctFromRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/rel/rules/RemoveIsNotDistinctFromRule.java b/core/src/main/java/org/eigenbase/rel/rules/RemoveIsNotDistinctFromRule.java
index f05bf9d..d415f2d 100644
--- a/core/src/main/java/org/eigenbase/rel/rules/RemoveIsNotDistinctFromRule.java
+++ b/core/src/main/java/org/eigenbase/rel/rules/RemoveIsNotDistinctFromRule.java
@@ -60,7 +60,7 @@ public final class RemoveIsNotDistinctFromRule extends RelOptRule {
             oldFilterRel.getCluster().getRexBuilder());
 
     RelNode newFilterRel =
-        CalcRel.createFilter(
+        RelOptUtil.createFilter(
             oldFilterRel.getChild(),
             oldFilterCond.accept(rewriteShuttle));
 

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/771ec544/core/src/main/java/org/eigenbase/rel/rules/RemoveTrivialProjectRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/rel/rules/RemoveTrivialProjectRule.java b/core/src/main/java/org/eigenbase/rel/rules/RemoveTrivialProjectRule.java
index 0fa6d4e..a191424 100644
--- a/core/src/main/java/org/eigenbase/rel/rules/RemoveTrivialProjectRule.java
+++ b/core/src/main/java/org/eigenbase/rel/rules/RemoveTrivialProjectRule.java
@@ -92,7 +92,7 @@ public class RemoveTrivialProjectRule extends RelOptRule {
   }
 
   public static boolean isIdentity(
-      List<RexNode> exps,
+      List<? extends RexNode> exps,
       RelDataType rowType,
       RelDataType childRowType) {
     List<RelDataTypeField> fields = rowType.getFieldList();

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/771ec544/core/src/main/java/org/eigenbase/rel/rules/SwapJoinRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/rel/rules/SwapJoinRule.java b/core/src/main/java/org/eigenbase/rel/rules/SwapJoinRule.java
index b1ec477..5d353e1 100644
--- a/core/src/main/java/org/eigenbase/rel/rules/SwapJoinRule.java
+++ b/core/src/main/java/org/eigenbase/rel/rules/SwapJoinRule.java
@@ -93,7 +93,7 @@ public class SwapJoinRule extends RelOptRule {
             join.getLeft(), joinType.swap(), join.isSemiJoinDone());
     final List<RexNode> exps =
         RelOptUtil.createSwappedJoinExprs(newJoin, join, true);
-    return CalcRel.createProject(
+    return RelOptUtil.createProject(
         newJoin,
         exps,
         join.getRowType().getFieldNames(),

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/771ec544/core/src/main/java/org/eigenbase/rel/rules/WindowedAggSplitterRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/rel/rules/WindowedAggSplitterRule.java b/core/src/main/java/org/eigenbase/rel/rules/WindowedAggSplitterRule.java
index a8737be..a23c203 100644
--- a/core/src/main/java/org/eigenbase/rel/rules/WindowedAggSplitterRule.java
+++ b/core/src/main/java/org/eigenbase/rel/rules/WindowedAggSplitterRule.java
@@ -117,7 +117,7 @@ public abstract class WindowedAggSplitterRule extends RelOptRule {
                           program.getCondition()));
                 }
                 if (!program.projectsOnlyIdentity()) {
-                  rel = CalcRel.createProject(
+                  rel = RelOptUtil.createProject(
                       rel,
                       Lists.transform(
                           program.getProjectList(),

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/771ec544/core/src/main/java/org/eigenbase/relopt/RelOptMaterialization.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/relopt/RelOptMaterialization.java b/core/src/main/java/org/eigenbase/relopt/RelOptMaterialization.java
index b2d8c70..74939b6 100644
--- a/core/src/main/java/org/eigenbase/relopt/RelOptMaterialization.java
+++ b/core/src/main/java/org/eigenbase/relopt/RelOptMaterialization.java
@@ -76,7 +76,7 @@ public class RelOptMaterialization {
                       starRelOptTable.getRowType().getFieldCount(),
                       0, 0, relOptTable.getRowType().getFieldCount());
 
-              return CalcRel.createProject(
+              return RelOptUtil.createProject(
                   new TableAccessRel(scan.getCluster(), starRelOptTable),
                   Mappings.asList(mapping.inverse()));
             }
@@ -158,7 +158,7 @@ public class RelOptMaterialization {
                           ((StarTable) leftTable).columnOffset(rightTable),
                           leftRelOptTable.getRowType().getFieldCount()));
               throw new Util.FoundOne(
-                  CalcRel.createProject(
+                  RelOptUtil.createProject(
                       new TableAccessRel(cluster, leftRelOptTable),
                       Mappings.asList(mapping.inverse())));
             }
@@ -168,7 +168,7 @@ public class RelOptMaterialization {
               Mappings.TargetMapping mapping =
                   Mappings.append(leftMapping, rightMapping);
               throw new Util.FoundOne(
-                  CalcRel.createProject(
+                  RelOptUtil.createProject(
                       new TableAccessRel(cluster, rightRelOptTable),
                       Mappings.asList(mapping.inverse())));
             }

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/771ec544/core/src/main/java/org/eigenbase/relopt/RelOptUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/relopt/RelOptUtil.java b/core/src/main/java/org/eigenbase/relopt/RelOptUtil.java
index c5e11c7..b898bbf 100644
--- a/core/src/main/java/org/eigenbase/relopt/RelOptUtil.java
+++ b/core/src/main/java/org/eigenbase/relopt/RelOptUtil.java
@@ -267,7 +267,7 @@ public abstract class RelOptUtil {
           RexUtil.composeConjunction(
               cluster.getRexBuilder(), conditions, true);
 
-      ret = CalcRel.createFilter(ret, conditionExp);
+      ret = createFilter(ret, conditionExp);
     }
 
     if (extraExpr != null) {
@@ -281,10 +281,8 @@ public abstract class RelOptUtil {
       // agg does not like no agg functions so just pretend it is
       // doing a min(TRUE)
 
-      ret =
-          CalcRel.createProject(
-              ret, Collections.<RexNode>singletonList(extraExpr), null);
-      List<RelDataType> argTypes =
+      ret = createProject(ret, ImmutableList.of(extraExpr), null);
+      final List<RelDataType> argTypes =
           ImmutableList.of(
               typeFactory.createSqlType(SqlTypeName.BOOLEAN));
 
@@ -295,14 +293,15 @@ public abstract class RelOptUtil {
               SqlMinMaxAggFunction.MINMAX_COMPARABLE);
 
       RelDataType returnType =
-          minFunction.inferReturnType(new AggregateRelBase.AggCallBinding(
-              typeFactory, minFunction, argTypes, 0));
+          minFunction.inferReturnType(
+              new AggregateRelBase.AggCallBinding(
+                  typeFactory, minFunction, argTypes, 0));
 
       final AggregateCall aggCall =
           new AggregateCall(
               minFunction,
               false,
-              Collections.singletonList(0),
+              ImmutableList.of(0),
               returnType,
               extraName);
 
@@ -311,7 +310,7 @@ public abstract class RelOptUtil {
               ret.getCluster(),
               ret,
               BitSets.of(),
-              Collections.singletonList(aggCall));
+              ImmutableList.of(aggCall));
     }
 
     return ret;
@@ -367,7 +366,7 @@ public abstract class RelOptUtil {
       final int projectedKeyCount = exprs.size();
       exprs.add(rexBuilder.makeLiteral(true));
 
-      ret = CalcRel.createProject(ret, exprs, null);
+      ret = createProject(ret, exprs, null);
 
       final List<RelDataType> argTypes =
           ImmutableList.of(typeFactory.createSqlType(SqlTypeName.BOOLEAN));
@@ -440,8 +439,19 @@ public abstract class RelOptUtil {
                   inputField.getIndex()),
               outputField.getName()));
     }
-    return CalcRel.createProject(
-        rel, Pair.left(renames), Pair.right(renames));
+    return createProject(rel, Pair.left(renames), Pair.right(renames));
+  }
+
+  /**
+   * Creates a relational expression which filters according to a given
+   * condition, returning the same fields as its input.
+   *
+   * @param child     Child relational expression
+   * @param condition Condition
+   * @return Relational expression
+   */
+  public static RelNode createFilter(RelNode child, RexNode condition) {
+    return new FilterRel(child.getCluster(), child, condition);
   }
 
   /**
@@ -468,7 +478,7 @@ public abstract class RelOptUtil {
     for (int i = 0; i < n; ++i) {
       int iField;
       if (fieldOrdinals != null) {
-        iField = fieldOrdinals[i].intValue();
+        iField = fieldOrdinals[i];
       } else {
         iField = i;
       }
@@ -495,7 +505,7 @@ public abstract class RelOptUtil {
       return rel;
     }
 
-    return CalcRel.createFilter(rel, condition);
+    return createFilter(rel, condition);
   }
 
   /**
@@ -579,7 +589,7 @@ public abstract class RelOptUtil {
           new AggregateCall(
               SqlStdOperatorTable.SINGLE_VALUE,
               false,
-              Collections.singletonList(i),
+              ImmutableList.of(i),
               returnType,
               null));
     }
@@ -1365,19 +1375,13 @@ public abstract class RelOptUtil {
     // added project if need to produce new keys than the original input
     // fields
     if (newLeftKeyCount > 0) {
-      leftRel =
-          CalcRel.createProject(
-              leftRel,
-              newLeftFields,
-              SqlValidatorUtil.uniquify(newLeftFieldNames));
+      leftRel = createProject(leftRel, newLeftFields,
+          SqlValidatorUtil.uniquify(newLeftFieldNames));
     }
 
     if (newRightKeyCount > 0) {
-      rightRel =
-          CalcRel.createProject(
-              rightRel,
-              newRightFields,
-              SqlValidatorUtil.uniquify(newRightFieldNames));
+      rightRel = createProject(rightRel, newRightFields,
+          SqlValidatorUtil.uniquify(newRightFieldNames));
     }
 
     inputRels[0] = leftRel;
@@ -1418,7 +1422,7 @@ public abstract class RelOptUtil {
       }
 
       // Create a project rel on the output of the join.
-      return CalcRel.createProject(
+      return createProject(
           joinRel,
           Pair.left(newProjects),
           Pair.right(newProjects));
@@ -1487,13 +1491,13 @@ public abstract class RelOptUtil {
     switch (kind) {
     case INSERT:
       return typeFactory.createStructType(
-          Collections.singletonList(
+          ImmutableList.of(
               Pair.of(
                   "ROWCOUNT",
                   typeFactory.createSqlType(SqlTypeName.BIGINT))));
     case EXPLAIN:
       return typeFactory.createStructType(
-          Collections.singletonList(
+          ImmutableList.of(
               Pair.of(
                   "PLAN",
                   typeFactory.createSqlType(
@@ -1717,7 +1721,7 @@ public abstract class RelOptUtil {
       return rel;
     }
     rel =
-        CalcRel.createRename(
+        createRename(
             rel,
             desiredRowType.getFieldNames());
     return rel;
@@ -2489,6 +2493,303 @@ public abstract class RelOptUtil {
         Mappings.apply3(mapping, rowType.getFieldList()));
   }
 
+  /**
+   * Creates a relational expression which projects a list of expressions.
+   *
+   * @param child         input relational expression
+   * @param exprList      list of expressions for the input columns
+   * @param fieldNameList aliases of the expressions, or null to generate
+   */
+  public static RelNode createProject(
+      RelNode child,
+      List<? extends RexNode> exprList,
+      List<String> fieldNameList) {
+    return createProject(child, exprList, fieldNameList, false);
+  }
+
+  /**
+   * Creates a relational expression which projects a list of (expression, name)
+   * pairs.
+   *
+   * @param child       input relational expression
+   * @param projectList list of (expression, name) pairs
+   * @param optimize    Whether to optimize
+   */
+  public static RelNode createProject(
+      RelNode child,
+      List<Pair<RexNode, String>> projectList,
+      boolean optimize) {
+    return createProject(child, Pair.left(projectList), Pair.right(projectList),
+        optimize);
+  }
+
+  /**
+   * Creates a relational expression that projects the given fields of the
+   * input.
+   *
+   * <p>Optimizes if the fields are the identity projection.</p>
+   *
+   * @param child   Input relational expression
+   * @param posList Source of each projected field
+   * @return Relational expression that projects given fields
+   */
+  public static RelNode createProject(final RelNode child,
+      final List<Integer> posList) {
+    return createProject(RelFactories.DEFAULT_PROJECT_FACTORY,
+        child, posList);
+  }
+
+  /**
+   * 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
+   * 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}.
+   *
+   * @param child      input relational expression
+   * @param exprs      list of expressions for the input columns
+   * @param fieldNames aliases of the expressions, or null to generate
+   * @param optimize   Whether to return <code>child</code> unchanged if the
+   *                   projections are trivial.
+   */
+  public static RelNode createProject(
+      RelNode child,
+      List<? extends RexNode> exprs,
+      List<String> fieldNames,
+      boolean optimize) {
+    final RelOptCluster cluster = child.getCluster();
+    final RexProgram program =
+        RexProgram.create(
+            child.getRowType(), exprs, null, fieldNames,
+            cluster.getRexBuilder());
+    final List<RelCollation> collationList =
+        program.getCollations(child.getCollationList());
+    final RelDataType rowType =
+        RexUtil.createStructType(
+            cluster.getTypeFactory(),
+            exprs,
+            fieldNames == null
+                ? null
+                : SqlValidatorUtil.uniquify(
+                    fieldNames, SqlValidatorUtil.F_SUGGESTER));
+    if (optimize
+        && RemoveTrivialProjectRule.isIdentity(exprs, rowType,
+            child.getRowType())) {
+      return child;
+    }
+    return new ProjectRel(cluster,
+        cluster.traitSetOf(collationList.isEmpty()
+            ? RelCollationImpl.EMPTY
+            : collationList.get(0)),
+        child,
+        exprs,
+        rowType,
+        ProjectRelBase.Flags.BOXED);
+  }
+
+  /**
+   * Returns a relational expression which has the same fields as the
+   * underlying expression, but the fields have different names.
+   *
+   * @param rel        Relational expression
+   * @param fieldNames Field names
+   * @return Renamed relational expression
+   */
+  public static RelNode createRename(
+      RelNode rel,
+      List<String> fieldNames) {
+    final List<RelDataTypeField> fields = rel.getRowType().getFieldList();
+    assert fieldNames.size() == fields.size();
+    final List<Pair<RexNode, String>> refs =
+        new AbstractList<Pair<RexNode, String>>() {
+          public int size() {
+            return fields.size();
+          }
+
+          public Pair<RexNode, String> get(int index) {
+            return RexInputRef.of2(index, fields);
+          }
+        };
+    return createProject(rel, refs, true);
+  }
+
+  /**
+   * Creates a relational expression which permutes the output fields of a
+   * relational expression according to a permutation.
+   *
+   * <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 permutation is the identity, returns the original relational
+   * expression.</li>
+   * </ul>
+   *
+   * <p>If a permutation is combined with its inverse, these optimizations
+   * would combine to remove them both.
+   *
+   * @param rel         Relational expression
+   * @param permutation Permutation to apply to fields
+   * @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 permutes its input fields
+   */
+  public static RelNode permute(
+      RelNode rel,
+      Permutation permutation,
+      List<String> fieldNames) {
+    if (permutation.isIdentity()) {
+      return rel;
+    }
+    if (rel instanceof CalcRel) {
+      CalcRel calcRel = (CalcRel) rel;
+      Permutation permutation1 = calcRel.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 (permutation1 != null) {
+        Permutation permutation2 = permutation.product(permutation1);
+        return permute(rel, permutation2, null);
+      }
+    }
+    final List<RelDataType> outputTypeList = new ArrayList<RelDataType>();
+    final List<String> outputNameList = new ArrayList<String>();
+    final List<RexNode> exprList = new ArrayList<RexNode>();
+    final List<RexLocalRef> projectRefList = new ArrayList<RexLocalRef>();
+    final List<RelDataTypeField> fields = rel.getRowType().getFieldList();
+    for (int i = 0; i < permutation.getTargetCount(); i++) {
+      int target = permutation.getTarget(i);
+      final RelDataTypeField targetField = fields.get(target);
+      outputTypeList.add(targetField.getType());
+      outputNameList.add(
+          ((fieldNames == null)
+              || (fieldNames.size() <= i)
+              || (fieldNames.get(i) == null)) ? targetField.getName()
+              : fieldNames.get(i));
+      exprList.add(
+          rel.getCluster().getRexBuilder().makeInputRef(
+              fields.get(i).getType(),
+              i));
+      final int source = permutation.getSource(i);
+      projectRefList.add(
+          new RexLocalRef(
+              source,
+              fields.get(source).getType()));
+    }
+    final RexProgram program =
+        new RexProgram(
+            rel.getRowType(),
+            exprList,
+            projectRefList,
+            null,
+            rel.getCluster().getTypeFactory().createStructType(
+                outputTypeList,
+                outputNameList));
+    return new CalcRel(
+        rel.getCluster(),
+        rel.getTraitSet(),
+        rel,
+        program.getOutputRowType(),
+        program,
+        ImmutableList.<RelCollation>of());
+  }
+
+  /**
+   * Creates a relational expression that projects the given fields of the
+   * input.
+   *
+   * <p>Optimizes if the fields are the identity projection.
+   *
+   * @param factory
+   *          ProjectFactory
+   * @param child
+   *          Input relational expression
+   * @param posList
+   *          Source of each projected field
+   * @return Relational expression that projects given fields
+   */
+  public static RelNode createProject(final RelFactories.ProjectFactory factory,
+      final RelNode child, final List<Integer> posList) {
+    if (Mappings.isIdentity(posList, child.getRowType().getFieldCount())) {
+      return child;
+    }
+    final RexBuilder rexBuilder = child.getCluster().getRexBuilder();
+    return factory.createProject(child,
+        new AbstractList<RexNode>() {
+          public int size() {
+            return posList.size();
+          }
+
+          public RexNode get(int index) {
+            final int pos = posList.get(index);
+            return rexBuilder.makeInputRef(child, pos);
+          }
+        },
+        null);
+  }
+
+  /**
+   * Creates a relational expression which projects the output fields of a
+   * relational expression according to a partial mapping.
+   *
+   * <p>A partial mapping is weaker than a permutation: every target has one
+   * source, but a source may have 0, 1 or more than one targets. Usually the
+   * result will have fewer fields than the source, unless some source fields
+   * are projected multiple times.
+   *
+   * <p>This method could optimize the result as {@link #permute} does, but
+   * 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 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
+   */
+  public static RelNode projectMapping(
+      RelNode rel,
+      Mapping mapping,
+      List<String> fieldNames,
+      RelFactories.ProjectFactory projectFactory) {
+    assert mapping.getMappingType().isSingleSource();
+    assert mapping.getMappingType().isMandatorySource();
+    if (mapping.isIdentity()) {
+      return rel;
+    }
+    final List<String> outputNameList = new ArrayList<String>();
+    final List<RexNode> exprList = new ArrayList<RexNode>();
+    final List<RelDataTypeField> fields = rel.getRowType().getFieldList();
+    final RexBuilder rexBuilder = rel.getCluster().getRexBuilder();
+    for (int i = 0; i < fields.size(); i++) {
+      exprList.add(rexBuilder.makeInputRef(rel, i));
+    }
+    for (int i = 0; i < mapping.getTargetCount(); i++) {
+      int source = mapping.getSource(i);
+      final RelDataTypeField sourceField = fields.get(source);
+      outputNameList.add(
+          ((fieldNames == null)
+              || (fieldNames.size() <= i)
+              || (fieldNames.get(i) == null)) ? sourceField.getName()
+              : fieldNames.get(i));
+    }
+    return projectFactory.createProject(rel, exprList, outputNameList);
+  }
+
   /** Policies for handling two- and three-valued boolean logic. */
   public enum Logic {
     /** Three-valued boolean logic. */

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/771ec544/core/src/main/java/org/eigenbase/relopt/SubstitutionVisitor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/relopt/SubstitutionVisitor.java b/core/src/main/java/org/eigenbase/relopt/SubstitutionVisitor.java
index 088933b..9dfd5c6 100644
--- a/core/src/main/java/org/eigenbase/relopt/SubstitutionVisitor.java
+++ b/core/src/main/java/org/eigenbase/relopt/SubstitutionVisitor.java
@@ -1491,7 +1491,8 @@ public class SubstitutionVisitor {
       return new MutableProject(rowType, input, projects);
     }
 
-    /** Equivalent to {@link CalcRel#createProject(RelNode, List, List)}
+    /** Equivalent to
+     * {@link RelOptUtil#createProject(org.eigenbase.rel.RelNode, java.util.List, java.util.List)}
      * for {@link MutableRel}. */
     static MutableRel of(MutableRel child, List<RexNode> exprList,
         List<String> fieldNameList) {
@@ -1790,7 +1791,9 @@ public class SubstitutionVisitor {
       }
     }
 
-    /** Based on {@link RemoveTrivialProjectRule#strip(ProjectRel)}. */
+    /** Based on
+     * {@link RemoveTrivialProjectRule#strip(org.eigenbase.rel.ProjectRelBase)}.
+     */
     public static MutableRel strip(MutableProject project) {
       return isTrivial(project) ? project.getChild() : project;
     }
@@ -1811,7 +1814,7 @@ public class SubstitutionVisitor {
       return true;
     }
 
-    /** Equivalent to {@link CalcRel#createProject(RelNode, List)}
+    /** Equivalent to {@link RelOptUtil#createProject(org.eigenbase.rel.RelNode, java.util.List)}
      * for {@link MutableRel}. */
     public static MutableRel createProject(final MutableRel child,
         final List<Integer> posList) {

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/771ec544/core/src/main/java/org/eigenbase/rex/RexProgram.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/rex/RexProgram.java b/core/src/main/java/org/eigenbase/rex/RexProgram.java
index 6122bec..d4ed909 100644
--- a/core/src/main/java/org/eigenbase/rex/RexProgram.java
+++ b/core/src/main/java/org/eigenbase/rex/RexProgram.java
@@ -165,12 +165,11 @@ public class RexProgram {
    */
   public static RexProgram create(
       RelDataType inputRowType,
-      List<RexNode> projectExprs,
+      List<? extends RexNode> projectExprs,
       RexNode conditionExpr,
       RelDataType outputRowType,
       RexBuilder rexBuilder) {
-    return create(
-        inputRowType, projectExprs, conditionExpr,
+    return create(inputRowType, projectExprs, conditionExpr,
         outputRowType.getFieldNames(), rexBuilder);
   }
 
@@ -187,7 +186,7 @@ public class RexProgram {
    */
   public static RexProgram create(
       RelDataType inputRowType,
-      List<RexNode> projectExprs,
+      List<? extends RexNode> projectExprs,
       RexNode conditionExpr,
       List<String> fieldNames,
       RexBuilder rexBuilder) {

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/771ec544/core/src/main/java/org/eigenbase/sql2rel/RelDecorrelator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/sql2rel/RelDecorrelator.java b/core/src/main/java/org/eigenbase/sql2rel/RelDecorrelator.java
index 95941bc..a818f48 100644
--- a/core/src/main/java/org/eigenbase/sql2rel/RelDecorrelator.java
+++ b/core/src/main/java/org/eigenbase/sql2rel/RelDecorrelator.java
@@ -450,7 +450,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
     // This ProjectRel will be what the old child maps to,
     // replacing any previous mapping from old child).
     RelNode newProjectRel =
-        CalcRel.createProject(newChildRel, projects, false);
+        RelOptUtil.createProject(newChildRel, projects, false);
 
     // update mappings:
     // oldChildRel ----> newChildRel
@@ -614,7 +614,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
     }
 
     RelNode newProjectRel =
-        CalcRel.createProject(newChildRel, projects, false);
+        RelOptUtil.createProject(newChildRel, projects, false);
 
     mapOldToNewRel.put(rel, newProjectRel);
     mapNewRelToMapOldToNewOutputPos.put(
@@ -699,7 +699,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
 
       if (!joinedInputRelSet.contains(newInputRel)) {
         RelNode projectRel =
-            CalcRel.createProject(
+            RelOptUtil.createProject(
                 newInputRel,
                 mapNewInputRelToOutputPos.get(newInputRel));
         RelNode distinctRel = RelOptUtil.createDistinctRel(projectRel);
@@ -862,7 +862,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
     // Replace the filter expression to reference output of the join
     // Map filter to the new filter over join
     RelNode newFilterRel =
-        CalcRel.createFilter(
+        RelOptUtil.createFilter(
             newChildRel,
             decorrelateExpr(rel.getCondition()));
 
@@ -1211,7 +1211,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
     }
 
     RelNode newProjRel =
-        CalcRel.createProject(joinRel, newProjExprs, false);
+        RelOptUtil.createProject(joinRel, newProjExprs, false);
 
     return newProjRel;
   }
@@ -1260,7 +1260,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
       newProjects.add(Pair.of(newProjExpr, pair.right));
     }
 
-    return CalcRel.createProject(corRel, newProjects, false);
+    return RelOptUtil.createProject(corRel, newProjects, false);
   }
 
   /**
@@ -1355,7 +1355,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
               field.e.getName()));
     }
     projects.addAll(additionalExprs);
-    return CalcRel.createProject(childRel, projects, false);
+    return RelOptUtil.createProject(childRel, projects, false);
   }
 
   //~ Inner Classes ----------------------------------------------------------
@@ -1724,7 +1724,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
       // projection that casts proj expr to a nullable type.
       final RelOptCluster cluster = projRel.getCluster();
       RelNode newProjRel =
-          CalcRel.createProject(aggRel,
+          RelOptUtil.createProject(aggRel,
               ImmutableList.of(
                   rexBuilder.makeCast(
                       cluster.getTypeFactory().createTypeWithNullability(
@@ -2249,7 +2249,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
           rexBuilder.makeInputRef(joinRel, nullIndicatorPos));
 
       RelNode joinOutputProjRel =
-          CalcRel.createProject(
+          RelOptUtil.createProject(
               joinRel,
               joinOutputProjExprs,
               null);
@@ -2308,7 +2308,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
               newAggOutputProjExpr));
 
       RelNode newAggOutputProjRel =
-          CalcRel.createProject(
+          RelOptUtil.createProject(
               newAggRel,
               newAggOutputProjExprList,
               null);
@@ -2362,7 +2362,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
           projects.add(RexInputRef.of2(projects.size(), fields));
         }
         aggOutputProjRel =
-            (ProjectRel) CalcRel.createProject(
+            (ProjectRel) RelOptUtil.createProject(
                 aggRel,
                 projects,
                 false);

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/771ec544/core/src/main/java/org/eigenbase/sql2rel/RelFieldTrimmer.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/sql2rel/RelFieldTrimmer.java b/core/src/main/java/org/eigenbase/sql2rel/RelFieldTrimmer.java
index d1b7c95..b0e3ccd 100644
--- a/core/src/main/java/org/eigenbase/sql2rel/RelFieldTrimmer.java
+++ b/core/src/main/java/org/eigenbase/sql2rel/RelFieldTrimmer.java
@@ -671,8 +671,8 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
       Mapping remaining = Mappings.divide(mapping, inputMapping);
 
       // Create a projection; does nothing if remaining is identity.
-      newInput = CalcRel.projectMapping(newInput, remaining, null,
-        projectFactory);
+      newInput = RelOptUtil.projectMapping(newInput, remaining, null,
+          projectFactory);
 
       if (input != newInput) {
         ++changeCount;

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/771ec544/core/src/main/java/org/eigenbase/sql2rel/RelStructuredTypeFlattener.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/sql2rel/RelStructuredTypeFlattener.java b/core/src/main/java/org/eigenbase/sql2rel/RelStructuredTypeFlattener.java
index 632c2d8..5fa73a9 100644
--- a/core/src/main/java/org/eigenbase/sql2rel/RelStructuredTypeFlattener.java
+++ b/core/src/main/java/org/eigenbase/sql2rel/RelStructuredTypeFlattener.java
@@ -137,7 +137,7 @@ public class RelStructuredTypeFlattener implements ReflectiveVisitor {
       // REVIEW jvs 23-Mar-2005:  How do we make sure that this
       // implementation stays in Java?  Fennel can't handle
       // structured types.
-      return CalcRel.createProject(
+      return RelOptUtil.createProject(
           flattened,
           structuringExps,
           root.getRowType().getFieldNames());
@@ -352,7 +352,7 @@ public class RelStructuredTypeFlattener implements ReflectiveVisitor {
 
   public void rewriteRel(FilterRel rel) {
     RelNode newRel =
-        CalcRel.createFilter(
+        RelOptUtil.createFilter(
             getNewForOldRel(rel.getChild()),
             flattenFieldAccesses(rel.getCondition()));
     setNewForOldRel(rel, newRel);
@@ -447,7 +447,7 @@ public class RelStructuredTypeFlattener implements ReflectiveVisitor {
         flattenedExpList,
         flattenedFieldNameList);
     RelNode newRel =
-        CalcRel.createProject(
+        RelOptUtil.createProject(
             getNewForOldRel(rel.getChild()),
             flattenedExpList,
             flattenedFieldNameList);

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/771ec544/core/src/main/java/org/eigenbase/sql2rel/SqlToRelConverter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/sql2rel/SqlToRelConverter.java b/core/src/main/java/org/eigenbase/sql2rel/SqlToRelConverter.java
index 0ef6e9f..6694895 100644
--- a/core/src/main/java/org/eigenbase/sql2rel/SqlToRelConverter.java
+++ b/core/src/main/java/org/eigenbase/sql2rel/SqlToRelConverter.java
@@ -806,7 +806,7 @@ public class SqlToRelConverter {
     // only allocate filter if the condition is not TRUE
     if (!convertedWhere.isAlwaysTrue()) {
       bb.setRoot(
-          CalcRel.createFilter(bb.root, convertedWhere),
+          RelOptUtil.createFilter(bb.root, convertedWhere),
           false);
     }
   }
@@ -1823,7 +1823,7 @@ public class SqlToRelConverter {
       final SqlNode node = ((SqlCall) from).operand(0);
       replaceSubqueries(bb, node, RelOptUtil.Logic.TRUE_FALSE_UNKNOWN);
       final RelNode childRel =
-          CalcRel.createProject(
+          RelOptUtil.createProject(
               (null != bb.root) ? bb.root : new OneRowRel(cluster),
               Collections.singletonList(bb.convertExpression(node)),
               Collections.singletonList(validator.deriveAlias(node, 0)),
@@ -2032,7 +2032,7 @@ public class SqlToRelConverter {
     if (!extraLeftExprs.isEmpty()) {
       final List<RelDataTypeField> fields =
           leftRel.getRowType().getFieldList();
-      leftRel = CalcRel.createProject(
+      leftRel = RelOptUtil.createProject(
           leftRel,
           new AbstractList<Pair<RexNode, String>>() {
             @Override
@@ -2059,7 +2059,7 @@ public class SqlToRelConverter {
       final List<RelDataTypeField> fields =
           rightRel.getRowType().getFieldList();
       final int newLeftCount = leftCount + extraLeftExprs.size();
-      rightRel = CalcRel.createProject(
+      rightRel = RelOptUtil.createProject(
           rightRel,
           new AbstractList<Pair<RexNode, String>>() {
             @Override
@@ -2513,7 +2513,7 @@ public class SqlToRelConverter {
 
       // Project the expressions required by agg and having.
       bb.setRoot(
-          CalcRel.createProject(
+          RelOptUtil.createProject(
               inputRel,
               preExprs,
               preNames,
@@ -2595,12 +2595,12 @@ public class SqlToRelConverter {
 
     // implement HAVING (we have already checked that it is non-trivial)
     if (havingExpr != null) {
-      bb.setRoot(CalcRel.createFilter(bb.root, havingExpr), false);
+      bb.setRoot(RelOptUtil.createFilter(bb.root, havingExpr), false);
     }
 
     // implement the SELECT list
     bb.setRoot(
-        CalcRel.createProject(
+        RelOptUtil.createProject(
             bb.root,
             selectExprs,
             selectNames,
@@ -2999,7 +2999,7 @@ public class SqlToRelConverter {
               sourceExps.get(i), field.getType()));
     }
 
-    return CalcRel.createProject(sourceRel, sourceExps, fieldNames, true);
+    return RelOptUtil.createProject(sourceRel, sourceExps, fieldNames, true);
   }
 
   private RexNode castNullLiteralIfNeeded(RexNode node, RelDataType type) {
@@ -3149,7 +3149,7 @@ public class SqlToRelConverter {
     }
 
     RelNode massagedRel =
-        CalcRel.createProject(joinRel, projects, null, true);
+        RelOptUtil.createProject(joinRel, projects, null, true);
 
     return new TableModificationRel(
         cluster,
@@ -3353,7 +3353,7 @@ public class SqlToRelConverter {
         }
 
         RelNode projRel =
-            CalcRel.createProject(
+            RelOptUtil.createProject(
                 new OneRowRel(cluster),
                 selectList,
                 fieldNameList);
@@ -3449,7 +3449,7 @@ public class SqlToRelConverter {
 
     RelNode inputRel = bb.root;
     bb.setRoot(
-        CalcRel.createProject(bb.root, exprs, fieldNames),
+        RelOptUtil.createProject(bb.root, exprs, fieldNames),
         false);
 
     assert bb.columnMonotonicities.isEmpty();
@@ -3565,7 +3565,7 @@ public class SqlToRelConverter {
               ? new OneRowRel(cluster)
               : tmpBb.root;
       unionRels.add(
-          CalcRel.createProject(
+          RelOptUtil.createProject(
               in,
               Pair.left(exps),
               Pair.right(exps),
@@ -3722,7 +3722,7 @@ public class SqlToRelConverter {
         }
 
         ProjectRel newLeftInput =
-            (ProjectRel) CalcRel.createProject(
+            (ProjectRel) RelOptUtil.createProject(
                 root,
                 newLeftInputExpr,
                 null,

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/771ec544/core/src/test/java/org/eigenbase/relopt/volcano/VolcanoPlannerTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/eigenbase/relopt/volcano/VolcanoPlannerTest.java b/core/src/test/java/org/eigenbase/relopt/volcano/VolcanoPlannerTest.java
index 5677ac3..0a060ef 100644
--- a/core/src/test/java/org/eigenbase/relopt/volcano/VolcanoPlannerTest.java
+++ b/core/src/test/java/org/eigenbase/relopt/volcano/VolcanoPlannerTest.java
@@ -29,6 +29,8 @@ import org.eigenbase.util.*;
 
 import net.hydromatic.optiq.rules.java.EnumerableConvention;
 
+import com.google.common.collect.ImmutableList;
+
 import org.junit.Ignore;
 import org.junit.Test;
 
@@ -227,10 +229,10 @@ public class VolcanoPlannerTest {
             "a");
     RexInputRef inputRef = RexInputRef.of(0, leafRel.getRowType());
     RelNode projectRel =
-        CalcRel.createProject(
+        RelOptUtil.createProject(
             leafRel,
-            Collections.<RexNode>singletonList(inputRef),
-            Collections.singletonList("this"));
+            ImmutableList.of(inputRef),
+            ImmutableList.of("this"));
     NoneSingleRel singleRel =
         new NoneSingleRel(
             cluster,