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:05 UTC

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

[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();