You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by jh...@apache.org on 2016/07/08 19:15:18 UTC

calcite git commit: [CALCITE-1297] RelBuilder should rename fields without creating an identity Project (Jark Wu) [Forced Update!]

Repository: calcite
Updated Branches:
  refs/heads/master d485eab4a -> 2193c6e6e (forced update)


[CALCITE-1297] RelBuilder should rename fields without creating an identity Project (Jark Wu)

Extend implementation to propagate field names through the group key
of an Aggregate. (Julian Hyde)

Close apache/calcite#251


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

Branch: refs/heads/master
Commit: 2193c6e6e88873dc4b63d8572eebeaa51c795c6c
Parents: a3bc0d8
Author: \u4e91\u90aa <wu...@alibaba-inc.com>
Authored: Fri Jul 8 12:58:07 2016 +0800
Committer: Julian Hyde <jh...@apache.org>
Committed: Fri Jul 8 12:15:09 2016 -0700

----------------------------------------------------------------------
 .../org/apache/calcite/tools/RelBuilder.java    | 97 +++++++++++++++-----
 .../org/apache/calcite/util/CompositeList.java  | 26 +++++-
 .../org/apache/calcite/test/RelBuilderTest.java | 45 +++++++++
 3 files changed, 139 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/2193c6e6/core/src/main/java/org/apache/calcite/tools/RelBuilder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/tools/RelBuilder.java b/core/src/main/java/org/apache/calcite/tools/RelBuilder.java
index d912ba2..30776d2 100644
--- a/core/src/main/java/org/apache/calcite/tools/RelBuilder.java
+++ b/core/src/main/java/org/apache/calcite/tools/RelBuilder.java
@@ -53,6 +53,8 @@ import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlOperator;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.validate.SqlValidatorUtil;
+import org.apache.calcite.util.CompositeList;
 import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.ImmutableIntList;
 import org.apache.calcite.util.Litmus;
@@ -235,19 +237,31 @@ public class RelBuilder {
   /** Returns the relational expression at the top of the stack, but does not
    * remove it. */
   public RelNode peek() {
-    return stack.peek().rel;
+    return peek_().rel;
+  }
+
+  private Frame peek_() {
+    return stack.peek();
   }
 
   /** Returns the relational expression {@code n} positions from the top of the
    * stack, but does not remove it. */
   public RelNode peek(int n) {
-    return Iterables.get(stack, n).rel;
+    return peek_(n).rel;
+  }
+
+  private Frame peek_(int n) {
+    return Iterables.get(stack, n);
   }
 
   /** Returns the relational expression {@code n} positions from the top of the
    * stack, but does not remove it. */
   public RelNode peek(int inputCount, int inputOrdinal) {
-    return peek(inputCount - 1 - inputOrdinal);
+    return peek_(inputCount, inputOrdinal).rel;
+  }
+
+  private Frame peek_(int inputCount, int inputOrdinal) {
+    return peek_(inputCount - 1 - inputOrdinal);
   }
 
   /** Returns the number of fields in all inputs before (to the left of)
@@ -307,14 +321,15 @@ public class RelBuilder {
    * @param fieldName Field name
    */
   public RexInputRef field(int inputCount, int inputOrdinal, String fieldName) {
-    final RelNode input = peek(inputCount, inputOrdinal);
-    final RelDataType rowType = input.getRowType();
-    final int ordinal = rowType.getFieldNames().indexOf(fieldName);
-    if (ordinal < 0) {
+    final Frame frame = peek_(inputCount, inputOrdinal);
+    final List<String> fieldNames = Pair.left(frame.fields());
+    int i = fieldNames.indexOf(fieldName);
+    if (i >= 0) {
+      return field(inputCount, inputOrdinal, i);
+    } else {
       throw new IllegalArgumentException("field [" + fieldName
-          + "] not found; input fields are: " + rowType.getFieldNames());
+          + "] not found; input fields are: " + fieldNames);
     }
-    return field(inputCount, inputOrdinal, ordinal);
   }
 
   /** Creates a reference to an input field by ordinal.
@@ -324,7 +339,7 @@ public class RelBuilder {
    * @param fieldOrdinal Field ordinal
    */
   public RexInputRef field(int fieldOrdinal) {
-    return field(1, 0, fieldOrdinal);
+    return (RexInputRef) field(1, 0, fieldOrdinal, false);
   }
 
   /** Creates a reference to a field of a given input relational expression
@@ -335,17 +350,32 @@ public class RelBuilder {
    * @param fieldOrdinal Field ordinal within input
    */
   public RexInputRef field(int inputCount, int inputOrdinal, int fieldOrdinal) {
-    final RelNode input = peek(inputCount, inputOrdinal);
+    return (RexInputRef) field(inputCount, inputOrdinal, fieldOrdinal, false);
+  }
+
+  /** As {@link #field(int, int, int)}, but if {@code alias} is true, the method
+   * may apply an alias to make sure that the field has the same name as in the
+   * input frame. If no alias is applied the expression is definitely a
+   * {@link RexInputRef}. */
+  private RexNode field(int inputCount, int inputOrdinal, int fieldOrdinal,
+      boolean alias) {
+    final Frame frame = peek_(inputCount, inputOrdinal);
+    final RelNode input = frame.rel;
     final RelDataType rowType = input.getRowType();
     if (fieldOrdinal < 0 || fieldOrdinal > rowType.getFieldCount()) {
       throw new IllegalArgumentException("field ordinal [" + fieldOrdinal
           + "] out of range; input fields are: " + rowType.getFieldNames());
     }
-    final RelDataType fieldType =
-        rowType.getFieldList().get(fieldOrdinal).getType();
+    final RelDataTypeField field = rowType.getFieldList().get(fieldOrdinal);
     final int offset = inputOffset(inputCount, inputOrdinal);
-    return cluster.getRexBuilder()
-        .makeInputRef(fieldType, offset + fieldOrdinal);
+    final RexInputRef ref = cluster.getRexBuilder()
+        .makeInputRef(field.getType(), offset + fieldOrdinal);
+    final RelDataTypeField aliasField = frame.fields().get(fieldOrdinal);
+    if (!alias || field.getName().equals(aliasField.getName())) {
+      return ref;
+    } else {
+      return alias(ref, aliasField.getName());
+    }
   }
 
   /** Creates a reference to a field of the current record which originated
@@ -416,7 +446,7 @@ public class RelBuilder {
   public ImmutableList<RexNode> fields(List<? extends Number> ordinals) {
     final ImmutableList.Builder<RexNode> nodes = ImmutableList.builder();
     for (Number ordinal : ordinals) {
-      RexNode node = field(ordinal.intValue());
+      RexNode node = field(1, 0, ordinal.intValue(), true);
       nodes.add(node);
     }
     return nodes.build();
@@ -785,14 +815,22 @@ public class RelBuilder {
       final String name2 = inferAlias(exprList, node);
       names.add(Util.first(name, name2));
     }
-    if (RexUtil.isIdentity(exprList, peek().getRowType())) {
-      return this;
-    }
     final RelDataType inputRowType = peek().getRowType();
-    if (RexUtil.isIdentity(exprList, inputRowType)
-        && names.equals(inputRowType.getFieldNames())) {
-      // Do not create an identity project if it does not rename any fields
-      return this;
+    if (RexUtil.isIdentity(exprList, inputRowType)) {
+      if (names.equals(inputRowType.getFieldNames())) {
+        // Do not create an identity project if it does not rename any fields
+        return this;
+      } else {
+        // create "virtual" row type for project only rename fields
+        final Frame frame = stack.pop();
+        final RelDataType rowType =
+            RexUtil.createStructType(cluster.getTypeFactory(), exprList,
+                names, SqlValidatorUtil.F_SUGGESTER);
+        stack.push(
+            new Frame(frame.rel,
+                ImmutableList.of(Pair.of(frame.right.get(0).left, rowType))));
+        return this;
+      }
     }
     final RelNode project =
         projectFactory.createProject(build(), ImmutableList.copyOf(exprList),
@@ -1110,7 +1148,7 @@ public class RelBuilder {
     final Frame pair = stack.pop();
     stack.push(
         new Frame(pair.rel,
-            ImmutableList.of(Pair.of(alias, pair.rel.getRowType()))));
+            ImmutableList.of(Pair.of(alias, pair.right.get(0).right))));
     return this;
   }
 
@@ -1535,6 +1573,13 @@ public class RelBuilder {
    * <p>Describes a previously created relational expression and
    * information about how table aliases map into its row type. */
   private static class Frame {
+    static final Function<Pair<String, RelDataType>, List<RelDataTypeField>> FN =
+        new Function<Pair<String, RelDataType>, List<RelDataTypeField>>() {
+          public List<RelDataTypeField> apply(Pair<String, RelDataType> input) {
+            return input.right.getFieldList();
+          }
+        };
+
     final RelNode rel;
     final ImmutableList<Pair<String, RelDataType>> right;
 
@@ -1556,6 +1601,10 @@ public class RelBuilder {
       }
       return null;
     }
+
+    List<RelDataTypeField> fields() {
+      return CompositeList.ofCopy(Iterables.transform(right, FN));
+    }
   }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/2193c6e6/core/src/main/java/org/apache/calcite/util/CompositeList.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/CompositeList.java b/core/src/main/java/org/apache/calcite/util/CompositeList.java
index 85ed62c..be38249 100644
--- a/core/src/main/java/org/apache/calcite/util/CompositeList.java
+++ b/core/src/main/java/org/apache/calcite/util/CompositeList.java
@@ -36,14 +36,14 @@ import java.util.List;
  * @param <T> Element type
  */
 public class CompositeList<T> extends AbstractList<T> {
-  private final ImmutableList<List<? extends T>> lists;
+  private final ImmutableList<List<T>> lists;
 
   /**
    * Creates a CompositeList.
    *
    * @param lists Constituent lists
    */
-  private CompositeList(ImmutableList<List<? extends T>> lists) {
+  private CompositeList(ImmutableList<List<T>> lists) {
     this.lists = lists;
   }
 
@@ -54,8 +54,22 @@ public class CompositeList<T> extends AbstractList<T> {
    * @param <T>   Element type
    * @return List consisting of all lists
    */
+  @SafeVarargs
   public static <T> CompositeList<T> of(List<? extends T>... lists) {
-    return new CompositeList<T>(ImmutableList.copyOf(lists));
+    //noinspection unchecked
+    return new CompositeList<T>((ImmutableList) ImmutableList.copyOf(lists));
+  }
+
+  /**
+   * Creates a CompositeList.
+   *
+   * @param lists Constituent lists
+   * @param <T>   Element type
+   * @return List consisting of all lists
+   */
+  public static <T> CompositeList<T> ofCopy(Iterable<List<T>> lists) {
+    final ImmutableList<List<T>> list = ImmutableList.copyOf(lists);
+    return new CompositeList<T>(list);
   }
 
   /**
@@ -89,7 +103,8 @@ public class CompositeList<T> extends AbstractList<T> {
    */
   public static <T> CompositeList<T> of(List<? extends T> list0,
       List<? extends T> list1) {
-    return new CompositeList<T>(ImmutableList.of(list0, list1));
+    //noinspection unchecked
+    return new CompositeList<T>((ImmutableList) ImmutableList.of(list0, list1));
   }
 
   /**
@@ -104,7 +119,8 @@ public class CompositeList<T> extends AbstractList<T> {
   public static <T> CompositeList<T> of(List<? extends T> list0,
       List<? extends T> list1,
       List<? extends T> list2) {
-    return new CompositeList<T>(ImmutableList.of(list0, list1, list2));
+    //noinspection unchecked
+    return new CompositeList<T>((ImmutableList) ImmutableList.of(list0, list1, list2));
   }
 
   public T get(int index) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/2193c6e6/core/src/test/java/org/apache/calcite/test/RelBuilderTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/RelBuilderTest.java b/core/src/test/java/org/apache/calcite/test/RelBuilderTest.java
index bb9f7b4..c79fbdc 100644
--- a/core/src/test/java/org/apache/calcite/test/RelBuilderTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RelBuilderTest.java
@@ -413,6 +413,51 @@ public class RelBuilderTest {
     assertThat(str(root), is(expected));
   }
 
+  /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-1297">[CALCITE-1297]
+   * RelBuilder does not translate identity projects even if they rename
+   * fields</a>. */
+  @Test public void testProjectIdentityWithFieldsRename() {
+    final RelBuilder builder = RelBuilder.create(config().build());
+    RelNode root =
+        builder.scan("DEPT")
+            .project(builder.alias(builder.field(0), "a"),
+                builder.alias(builder.field(1), "b"),
+                builder.alias(builder.field(2), "c"))
+            .as("t1")
+            .project(builder.field("a"),
+                builder.field("t1", "c"))
+            .build();
+    final String expected = "LogicalProject(DEPTNO=[$0], LOC=[$2])\n"
+        + "  LogicalTableScan(table=[[scott, DEPT]])\n";
+    assertThat(str(root), is(expected));
+  }
+
+  /** Variation on {@link #testProjectIdentityWithFieldsRename}: don't use a
+   * table alias, and make sure the field names propagate through a filter. */
+  @Test public void testProjectIdentityWithFieldsRenameFilter() {
+    final RelBuilder builder = RelBuilder.create(config().build());
+    RelNode root =
+        builder.scan("DEPT")
+            .project(builder.alias(builder.field(0), "a"),
+                builder.alias(builder.field(1), "b"),
+                builder.alias(builder.field(2), "c"))
+            .filter(
+                builder.call(SqlStdOperatorTable.EQUALS,
+                    builder.field("a"),
+                    builder.literal(20)))
+            .aggregate(builder.groupKey(0, 1, 2))
+            .project(builder.field("c"),
+                builder.field("a"))
+            .build();
+    final String expected = "LogicalProject(c=[$2], a=[$0])\n"
+        + "  LogicalAggregate(group=[{3, 4, 5}])\n"
+        + "    LogicalProject(DEPTNO=[$0], DNAME=[$1], LOC=[$2], a=[$0], b=[$1], c=[$2])\n"
+        + "      LogicalFilter(condition=[=($0, 20)])\n"
+        + "        LogicalTableScan(table=[[scott, DEPT]])\n";
+    assertThat(str(root), is(expected));
+  }
+
   @Test public void testProjectLeadingEdge() {
     final RelBuilder builder = RelBuilder.create(config().build());
     RelNode root =