You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by hy...@apache.org on 2019/06/30 20:27:57 UTC

[calcite] branch master updated (78251a1 -> a750bbc)

This is an automated email from the ASF dual-hosted git repository.

hyuan pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/calcite.git.


    from 78251a1  [CALCITE-2801] Check input type in AggregateUnionAggregateRule when remove the bottom Aggregate (Hequn Cheng)
     new 3cf36e0  [CALCITE-3151] RexCall's Monotonicity is not considered in determining a Calc's collation
     new cebaa2d  address reviews
     new 82a1226  fix checkstyle
     new a750bbc  fix checkstyle

The 4 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../calcite/rel/metadata/RelMdCollation.java       |  9 +++-
 .../org/apache/calcite/test/RelOptRulesTest.java   | 53 ++++++++++++++++++++++
 2 files changed, 61 insertions(+), 1 deletion(-)


[calcite] 01/04: [CALCITE-3151] RexCall's Monotonicity is not considered in determining a Calc's collation

Posted by hy...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

hyuan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/calcite.git

commit 3cf36e04355712572014e1c8f9bf24531ec5772c
Author: Zhenghua Gao <do...@gmail.com>
AuthorDate: Thu Jun 27 19:55:52 2019 +0800

    [CALCITE-3151] RexCall's Monotonicity is not considered in determining a Calc's collation
---
 .../calcite/rel/metadata/RelMdCollation.java       | 129 +++++++++++----------
 .../org/apache/calcite/test/RelOptRulesTest.java   |  49 ++++++++
 2 files changed, 119 insertions(+), 59 deletions(-)

diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdCollation.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdCollation.java
index 6da79ce..2b6bbb5 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdCollation.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdCollation.java
@@ -65,6 +65,7 @@ import java.util.Map;
 import java.util.Objects;
 import java.util.SortedSet;
 import java.util.TreeSet;
+import java.util.stream.Collectors;
 
 /**
  * RelMdCollation supplies a default implementation of
@@ -240,70 +241,19 @@ public class RelMdCollation
    * {@link org.apache.calcite.rel.core.Calc}'s collation. */
   public static List<RelCollation> calc(RelMetadataQuery mq, RelNode input,
       RexProgram program) {
-    return program.getCollations(mq.collations(input));
+    final List<RexNode> projects =
+        program
+            .getProjectList()
+            .stream()
+            .map((p) -> program.expandLocalRef(p))
+            .collect(Collectors.toList());
+    return enumrableCalc(mq, input, projects);
   }
 
   /** Helper method to determine a {@link Project}'s collation. */
   public static List<RelCollation> project(RelMetadataQuery mq,
       RelNode input, List<? extends RexNode> projects) {
-    final SortedSet<RelCollation> collations = new TreeSet<>();
-    final List<RelCollation> inputCollations = mq.collations(input);
-    if (inputCollations == null || inputCollations.isEmpty()) {
-      return ImmutableList.of();
-    }
-    final Multimap<Integer, Integer> targets = LinkedListMultimap.create();
-    final Map<Integer, SqlMonotonicity> targetsWithMonotonicity =
-        new HashMap<>();
-    for (Ord<RexNode> project : Ord.<RexNode>zip(projects)) {
-      if (project.e instanceof RexInputRef) {
-        targets.put(((RexInputRef) project.e).getIndex(), project.i);
-      } else if (project.e instanceof RexCall) {
-        final RexCall call = (RexCall) project.e;
-        final RexCallBinding binding =
-            RexCallBinding.create(input.getCluster().getTypeFactory(), call, inputCollations);
-        targetsWithMonotonicity.put(project.i, call.getOperator().getMonotonicity(binding));
-      }
-    }
-    final List<RelFieldCollation> fieldCollations = new ArrayList<>();
-  loop:
-    for (RelCollation ic : inputCollations) {
-      if (ic.getFieldCollations().isEmpty()) {
-        continue;
-      }
-      fieldCollations.clear();
-      for (RelFieldCollation ifc : ic.getFieldCollations()) {
-        final Collection<Integer> integers = targets.get(ifc.getFieldIndex());
-        if (integers.isEmpty()) {
-          continue loop; // cannot do this collation
-        }
-        fieldCollations.add(ifc.withFieldIndex(integers.iterator().next()));
-      }
-      assert !fieldCollations.isEmpty();
-      collations.add(RelCollations.of(fieldCollations));
-    }
-
-    final List<RelFieldCollation> fieldCollationsForRexCalls =
-        new ArrayList<>();
-    for (Map.Entry<Integer, SqlMonotonicity> entry
-        : targetsWithMonotonicity.entrySet()) {
-      final SqlMonotonicity value = entry.getValue();
-      switch (value) {
-      case NOT_MONOTONIC:
-      case CONSTANT:
-        break;
-      default:
-        fieldCollationsForRexCalls.add(
-            new RelFieldCollation(entry.getKey(),
-                RelFieldCollation.Direction.of(value)));
-        break;
-      }
-    }
-
-    if (!fieldCollationsForRexCalls.isEmpty()) {
-      collations.add(RelCollations.of(fieldCollationsForRexCalls));
-    }
-
-    return ImmutableList.copyOf(collations);
+    return enumrableCalc(mq, input, projects);
   }
 
   /** Helper method to determine a
@@ -478,6 +428,67 @@ public class RelMdCollation
     }
     return ImmutableList.of();
   }
+
+  private static List<RelCollation> enumrableCalc(RelMetadataQuery mq,
+      RelNode input, List<? extends RexNode> projects) {
+    final SortedSet<RelCollation> collations = new TreeSet<>();
+    final List<RelCollation> inputCollations = mq.collations(input);
+    if (inputCollations == null || inputCollations.isEmpty()) {
+      return ImmutableList.of();
+    }
+    final Multimap<Integer, Integer> targets = LinkedListMultimap.create();
+    final Map<Integer, SqlMonotonicity> targetsWithMonotonicity =
+        new HashMap<>();
+    for (Ord<RexNode> project : Ord.<RexNode>zip(projects)) {
+      if (project.e instanceof RexInputRef) {
+        targets.put(((RexInputRef) project.e).getIndex(), project.i);
+      } else if (project.e instanceof RexCall) {
+        final RexCall call = (RexCall) project.e;
+        final RexCallBinding binding =
+            RexCallBinding.create(input.getCluster().getTypeFactory(), call, inputCollations);
+        targetsWithMonotonicity.put(project.i, call.getOperator().getMonotonicity(binding));
+      }
+    }
+    final List<RelFieldCollation> fieldCollations = new ArrayList<>();
+    loop:
+    for (RelCollation ic : inputCollations) {
+      if (ic.getFieldCollations().isEmpty()) {
+        continue;
+      }
+      fieldCollations.clear();
+      for (RelFieldCollation ifc : ic.getFieldCollations()) {
+        final Collection<Integer> integers = targets.get(ifc.getFieldIndex());
+        if (integers.isEmpty()) {
+          continue loop; // cannot do this collation
+        }
+        fieldCollations.add(ifc.withFieldIndex(integers.iterator().next()));
+      }
+      assert !fieldCollations.isEmpty();
+      collations.add(RelCollations.of(fieldCollations));
+    }
+
+    final List<RelFieldCollation> fieldCollationsForRexCalls =
+        new ArrayList<>();
+    for (Map.Entry<Integer, SqlMonotonicity> entry : targetsWithMonotonicity.entrySet()) {
+      final SqlMonotonicity value = entry.getValue();
+      switch (value) {
+      case NOT_MONOTONIC:
+      case CONSTANT:
+        break;
+      default:
+        fieldCollationsForRexCalls.add(
+            new RelFieldCollation(entry.getKey(),
+                RelFieldCollation.Direction.of(value)));
+        break;
+      }
+    }
+
+    if (!fieldCollationsForRexCalls.isEmpty()) {
+      collations.add(RelCollations.of(fieldCollationsForRexCalls));
+    }
+
+    return ImmutableList.copyOf(collations);
+  }
 }
 
 // End RelMdCollation.java
diff --git a/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java b/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
index 2e33132..73a1258 100644
--- a/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
@@ -131,13 +131,18 @@ import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rex.RexCall;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.runtime.Hook;
+import org.apache.calcite.sql.SqlFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlOperatorBinding;
 import org.apache.calcite.sql.SqlSpecialOperator;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.type.OperandTypes;
 import org.apache.calcite.sql.type.ReturnTypes;
 import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.validate.SqlMonotonicity;
 import org.apache.calcite.sql.validate.SqlValidator;
 import org.apache.calcite.sql2rel.SqlToRelConverter;
 import org.apache.calcite.test.catalog.MockCatalogReader;
@@ -168,6 +173,7 @@ import static org.apache.calcite.plan.RelOptRule.operand;
 import static org.apache.calcite.plan.RelOptRule.operandJ;
 
 import static org.hamcrest.CoreMatchers.notNullValue;
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
 
 /**
@@ -5620,6 +5626,49 @@ public class RelOptRulesTest extends RelOptTestBase {
         .withPre(preProgram).with(program).check();
   }
 
+  @Test public void testMonotonicityUDF() throws Exception {
+    final SqlFunction monotonicityFun =
+        new SqlFunction("MONOFUN", SqlKind.OTHER_FUNCTION, ReturnTypes.BIGINT, null,
+            OperandTypes.NILADIC, SqlFunctionCategory.USER_DEFINED_FUNCTION) {
+          @Override public boolean isDeterministic() {
+            return false;
+          }
+
+          @Override public SqlMonotonicity getMonotonicity(SqlOperatorBinding call) {
+            return SqlMonotonicity.INCREASING;
+          }
+        };
+
+    // Build a tree equivalent to the SQL
+    //  SELECT sal, MONOFUN() AS n FROM emp
+    final RelBuilder builder =
+        RelBuilder.create(RelBuilderTest.config().build());
+    final RelNode root =
+        builder.scan("EMP")
+            .project(builder.field("SAL"),
+                builder.alias(builder.call(monotonicityFun), "M"))
+            .build();
+
+    HepProgram preProgram = new HepProgramBuilder().build();
+    HepPlanner prePlanner = new HepPlanner(preProgram);
+    prePlanner.setRoot(root);
+    final RelNode relBefore = prePlanner.findBestExp();
+    final RelCollation collationBefore =
+        relBefore.getTraitSet().getTrait(RelCollationTraitDef.INSTANCE);
+
+    HepProgram hepProgram = new HepProgramBuilder()
+        .addRuleInstance(ProjectToCalcRule.INSTANCE)
+        .build();
+
+    HepPlanner hepPlanner = new HepPlanner(hepProgram);
+    hepPlanner.setRoot(root);
+    final RelNode relAfter = hepPlanner.findBestExp();
+    final RelCollation collationAfter =
+        relAfter.getTraitSet().getTrait(RelCollationTraitDef.INSTANCE);
+
+    assertEquals(collationBefore, collationAfter);
+  }
+
   /**
    * Custom implementation of {@link Filter} for use
    * in test case to verify that {@link FilterMultiJoinMergeRule}


[calcite] 04/04: fix checkstyle

Posted by hy...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

hyuan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/calcite.git

commit a750bbc62840047f178055f4149ea2049e1f2fd4
Author: Zhenghua Gao <do...@gmail.com>
AuthorDate: Fri Jun 28 13:39:24 2019 +0800

    fix checkstyle
---
 core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java b/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
index 79fa2bf..337ce66 100644
--- a/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
@@ -5627,7 +5627,7 @@ public class RelOptRulesTest extends RelOptTestBase {
   }
 
   /** Test case for
-   * <a href="https://issues.apache.org/jira/browse/CALCITE-3151">[CALCITE-3151
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-3151">[CALCITE-3151]
    * RexCall's Monotonicity is not considered in determining a Calc's collation</a>
    */
   @Test public void testMonotonicityUDF() throws Exception {


[calcite] 02/04: address reviews

Posted by hy...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

hyuan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/calcite.git

commit cebaa2d91d88e59f44123f2a2344103996054ca4
Author: Zhenghua Gao <do...@gmail.com>
AuthorDate: Fri Jun 28 12:12:30 2019 +0800

    address reviews
---
 .../calcite/rel/metadata/RelMdCollation.java       | 121 ++++++++++-----------
 .../org/apache/calcite/test/RelOptRulesTest.java   |   6 +-
 2 files changed, 63 insertions(+), 64 deletions(-)

diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdCollation.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdCollation.java
index 2b6bbb5..7ac26e4 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdCollation.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdCollation.java
@@ -247,13 +247,69 @@ public class RelMdCollation
             .stream()
             .map((p) -> program.expandLocalRef(p))
             .collect(Collectors.toList());
-    return enumrableCalc(mq, input, projects);
+    return project(mq, input, projects);
   }
 
   /** Helper method to determine a {@link Project}'s collation. */
   public static List<RelCollation> project(RelMetadataQuery mq,
       RelNode input, List<? extends RexNode> projects) {
-    return enumrableCalc(mq, input, projects);
+    final SortedSet<RelCollation> collations = new TreeSet<>();
+    final List<RelCollation> inputCollations = mq.collations(input);
+    if (inputCollations == null || inputCollations.isEmpty()) {
+      return ImmutableList.of();
+    }
+    final Multimap<Integer, Integer> targets = LinkedListMultimap.create();
+    final Map<Integer, SqlMonotonicity> targetsWithMonotonicity =
+        new HashMap<>();
+    for (Ord<RexNode> project : Ord.<RexNode>zip(projects)) {
+      if (project.e instanceof RexInputRef) {
+        targets.put(((RexInputRef) project.e).getIndex(), project.i);
+      } else if (project.e instanceof RexCall) {
+        final RexCall call = (RexCall) project.e;
+        final RexCallBinding binding =
+            RexCallBinding.create(input.getCluster().getTypeFactory(), call, inputCollations);
+        targetsWithMonotonicity.put(project.i, call.getOperator().getMonotonicity(binding));
+      }
+    }
+    final List<RelFieldCollation> fieldCollations = new ArrayList<>();
+    loop:
+    for (RelCollation ic : inputCollations) {
+      if (ic.getFieldCollations().isEmpty()) {
+        continue;
+      }
+      fieldCollations.clear();
+      for (RelFieldCollation ifc : ic.getFieldCollations()) {
+        final Collection<Integer> integers = targets.get(ifc.getFieldIndex());
+        if (integers.isEmpty()) {
+          continue loop; // cannot do this collation
+        }
+        fieldCollations.add(ifc.withFieldIndex(integers.iterator().next()));
+      }
+      assert !fieldCollations.isEmpty();
+      collations.add(RelCollations.of(fieldCollations));
+    }
+
+    final List<RelFieldCollation> fieldCollationsForRexCalls =
+            new ArrayList<>();
+    for (Map.Entry<Integer, SqlMonotonicity> entry : targetsWithMonotonicity.entrySet()) {
+      final SqlMonotonicity value = entry.getValue();
+      switch (value) {
+      case NOT_MONOTONIC:
+      case CONSTANT:
+        break;
+      default:
+        fieldCollationsForRexCalls.add(
+            new RelFieldCollation(entry.getKey(),
+                RelFieldCollation.Direction.of(value)));
+        break;
+      }
+    }
+
+    if (!fieldCollationsForRexCalls.isEmpty()) {
+      collations.add(RelCollations.of(fieldCollationsForRexCalls));
+    }
+
+    return ImmutableList.copyOf(collations);
   }
 
   /** Helper method to determine a
@@ -428,67 +484,6 @@ public class RelMdCollation
     }
     return ImmutableList.of();
   }
-
-  private static List<RelCollation> enumrableCalc(RelMetadataQuery mq,
-      RelNode input, List<? extends RexNode> projects) {
-    final SortedSet<RelCollation> collations = new TreeSet<>();
-    final List<RelCollation> inputCollations = mq.collations(input);
-    if (inputCollations == null || inputCollations.isEmpty()) {
-      return ImmutableList.of();
-    }
-    final Multimap<Integer, Integer> targets = LinkedListMultimap.create();
-    final Map<Integer, SqlMonotonicity> targetsWithMonotonicity =
-        new HashMap<>();
-    for (Ord<RexNode> project : Ord.<RexNode>zip(projects)) {
-      if (project.e instanceof RexInputRef) {
-        targets.put(((RexInputRef) project.e).getIndex(), project.i);
-      } else if (project.e instanceof RexCall) {
-        final RexCall call = (RexCall) project.e;
-        final RexCallBinding binding =
-            RexCallBinding.create(input.getCluster().getTypeFactory(), call, inputCollations);
-        targetsWithMonotonicity.put(project.i, call.getOperator().getMonotonicity(binding));
-      }
-    }
-    final List<RelFieldCollation> fieldCollations = new ArrayList<>();
-    loop:
-    for (RelCollation ic : inputCollations) {
-      if (ic.getFieldCollations().isEmpty()) {
-        continue;
-      }
-      fieldCollations.clear();
-      for (RelFieldCollation ifc : ic.getFieldCollations()) {
-        final Collection<Integer> integers = targets.get(ifc.getFieldIndex());
-        if (integers.isEmpty()) {
-          continue loop; // cannot do this collation
-        }
-        fieldCollations.add(ifc.withFieldIndex(integers.iterator().next()));
-      }
-      assert !fieldCollations.isEmpty();
-      collations.add(RelCollations.of(fieldCollations));
-    }
-
-    final List<RelFieldCollation> fieldCollationsForRexCalls =
-        new ArrayList<>();
-    for (Map.Entry<Integer, SqlMonotonicity> entry : targetsWithMonotonicity.entrySet()) {
-      final SqlMonotonicity value = entry.getValue();
-      switch (value) {
-      case NOT_MONOTONIC:
-      case CONSTANT:
-        break;
-      default:
-        fieldCollationsForRexCalls.add(
-            new RelFieldCollation(entry.getKey(),
-                RelFieldCollation.Direction.of(value)));
-        break;
-      }
-    }
-
-    if (!fieldCollationsForRexCalls.isEmpty()) {
-      collations.add(RelCollations.of(fieldCollationsForRexCalls));
-    }
-
-    return ImmutableList.copyOf(collations);
-  }
 }
 
 // End RelMdCollation.java
diff --git a/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java b/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
index 73a1258..79fa2bf 100644
--- a/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
@@ -5626,6 +5626,10 @@ public class RelOptRulesTest extends RelOptTestBase {
         .withPre(preProgram).with(program).check();
   }
 
+  /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-3151">[CALCITE-3151
+   * RexCall's Monotonicity is not considered in determining a Calc's collation</a>
+   */
   @Test public void testMonotonicityUDF() throws Exception {
     final SqlFunction monotonicityFun =
         new SqlFunction("MONOFUN", SqlKind.OTHER_FUNCTION, ReturnTypes.BIGINT, null,
@@ -5640,7 +5644,7 @@ public class RelOptRulesTest extends RelOptTestBase {
         };
 
     // Build a tree equivalent to the SQL
-    //  SELECT sal, MONOFUN() AS n FROM emp
+    // SELECT sal, MONOFUN() AS n FROM emp
     final RelBuilder builder =
         RelBuilder.create(RelBuilderTest.config().build());
     final RelNode root =


[calcite] 03/04: fix checkstyle

Posted by hy...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

hyuan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/calcite.git

commit 82a12266d8ecb69c1675ddeb5705b2b22b1afe19
Author: Zhenghua Gao <do...@gmail.com>
AuthorDate: Fri Jun 28 12:16:19 2019 +0800

    fix checkstyle
---
 .../main/java/org/apache/calcite/rel/metadata/RelMdCollation.java  | 7 ++++---
 1 file changed, 4 insertions(+), 3 deletions(-)

diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdCollation.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdCollation.java
index 7ac26e4..b669229 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdCollation.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdCollation.java
@@ -272,7 +272,7 @@ public class RelMdCollation
       }
     }
     final List<RelFieldCollation> fieldCollations = new ArrayList<>();
-    loop:
+  loop:
     for (RelCollation ic : inputCollations) {
       if (ic.getFieldCollations().isEmpty()) {
         continue;
@@ -290,8 +290,9 @@ public class RelMdCollation
     }
 
     final List<RelFieldCollation> fieldCollationsForRexCalls =
-            new ArrayList<>();
-    for (Map.Entry<Integer, SqlMonotonicity> entry : targetsWithMonotonicity.entrySet()) {
+        new ArrayList<>();
+    for (Map.Entry<Integer, SqlMonotonicity> entry
+        : targetsWithMonotonicity.entrySet()) {
       final SqlMonotonicity value = entry.getValue();
       switch (value) {
       case NOT_MONOTONIC: