You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by GitBox <gi...@apache.org> on 2020/01/10 04:33:12 UTC

[GitHub] [calcite] cshuo opened a new pull request #1741: [CALCITE-3721] Filter of distinct aggregate call is lost after applyi…

cshuo opened a new pull request #1741: [CALCITE-3721] Filter of distinct aggregate call is lost after applyi…
URL: https://github.com/apache/calcite/pull/1741
 
 
   …ng `AggregateExpandDistinctAggregatesRule`
   
   In `AggregateExpandDistinctAggregatesRule`, when the distinct aggregate call is rewriting using 
   groupingSets, the filter of the distinct aggregate call itself is lost unexpected.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [calcite] vlsi commented on a change in pull request #1741: [CALCITE-3721] Filter of distinct aggregate call is lost after applyi…

Posted by GitBox <gi...@apache.org>.
vlsi commented on a change in pull request #1741: [CALCITE-3721] Filter of distinct aggregate call is lost after applyi…
URL: https://github.com/apache/calcite/pull/1741#discussion_r365345915
 
 

 ##########
 File path: core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
 ##########
 @@ -1474,6 +1474,27 @@ private void basePushFilterPastAggWithGroupingSets(boolean unchanged)
     sql(sql).with(program).check();
   }
 
+  @Test public void testDistinctWithFilterWithoutGroupBy() {
+    final String sql = "SELECT SUM(comm), COUNT(DISTINCT comm),\n"
+        + "COUNT(DISTINCT sal) FILTER (WHERE sal > 1000)\n"
+        + "FROM emp";
+    HepProgram program = new HepProgramBuilder()
+        .addRuleInstance(AggregateExpandDistinctAggregatesRule.INSTANCE)
+        .build();
+    sql(sql).with(program).check();
+  }
+
+  @Test public void testDistinctWithFilterAndGroupBy() {
+    final String sql = "SELECT deptno, SUM(comm), COUNT(DISTINCT comm),\n"
+        + "COUNT(DISTINCT sal) FILTER (WHERE sal > 1000)\n"
 
 Review comment:
   Is this a minimal reproducible example?
   
   Frankly speaking, it is really hard to follow the code, and it is sad it is the most trivial example in the PR.
   
   I would prefer to have very simple example that makes it clear what the rule produces. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [calcite] danny0405 commented on a change in pull request #1741: [CALCITE-3721] Filter of distinct aggregate call is lost after applyi…

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #1741: [CALCITE-3721] Filter of distinct aggregate call is lost after applyi…
URL: https://github.com/apache/calcite/pull/1741#discussion_r366251702
 
 

 ##########
 File path: core/src/main/java/org/apache/calcite/rel/rules/AggregateExpandDistinctAggregatesRule.java
 ##########
 @@ -432,31 +442,43 @@ private void rewriteUsingGroupingSets(RelOptRuleCall call,
     relBuilder.push(aggregate.getInput());
     final int groupCount = fullGroupSet.cardinality();
 
-    final Map<ImmutableBitSet, Integer> filters = new LinkedHashMap<>();
-    final int z = groupCount + distinctAggCalls.size();
+    // get the base ordinal for filter args for different groupSet.
+    final Map<Pair<ImmutableBitSet, Integer>, Integer> filters = new LinkedHashMap<>();
+    int z = groupCount + distinctAggCalls.size();
+    for (ImmutableBitSet groupSet: groupSets) {
+      Set<Integer> filterArgList = distinctFilterArgMap.get(groupSet);
+      for (Integer filterArg: filterArgList) {
+        filters.put(Pair.of(groupSet, filterArg), z);
+        z += 1;
+      }
+    }
+
     distinctAggCalls.add(
         AggregateCall.create(SqlStdOperatorTable.GROUPING, false, false, false,
             ImmutableIntList.copyOf(fullGroupSet), -1, RelCollations.EMPTY,
             groupSets.size(), relBuilder.peek(), null, "$g"));
-    for (Ord<ImmutableBitSet> groupSet : Ord.zip(groupSets)) {
-      filters.put(groupSet.e, z + groupSet.i);
-    }
-
     relBuilder.aggregate(relBuilder.groupKey(fullGroupSet, groupSets),
         distinctAggCalls);
-    final RelNode distinct = relBuilder.peek();
 
     // GROUPING returns an integer (0 or 1). Add a project to convert those
     // values to BOOLEAN.
     if (!filters.isEmpty()) {
       final List<RexNode> nodes = new ArrayList<>(relBuilder.fields());
       final RexNode nodeZ = nodes.remove(nodes.size() - 1);
-      for (Map.Entry<ImmutableBitSet, Integer> entry : filters.entrySet()) {
-        final long v = groupValue(fullGroupSet, entry.getKey());
+      for (Map.Entry<Pair<ImmutableBitSet, Integer>, Integer> entry : filters.entrySet()) {
+        final long v = groupValue(fullGroupSet, entry.getKey().left);
+        int distinctFilterArg = remap(fullGroupSet, entry.getKey().right);
+        RexNode expr = relBuilder.equals(nodeZ, relBuilder.literal(v));
+        if (distinctFilterArg > -1) {
+          // merge the filter of the distinct aggregate call itself.
+          RexBuilder rexBuilder = aggregate.getCluster().getRexBuilder();
+          expr = relBuilder.and(expr,
+              rexBuilder.makeCall(SqlStdOperatorTable.IS_TRUE,
+                  relBuilder.field(distinctFilterArg)));
+        }
         nodes.add(
-            relBuilder.alias(
-                relBuilder.equals(nodeZ, relBuilder.literal(v)),
-                "$g_" + v));
+            relBuilder.alias(expr,
+            "$g_" + v + (distinctFilterArg < 0 ? "" : "_" + distinctFilterArg)));
       }
 
 Review comment:
   I would suggest to use `$g_v_f_distinctFilterArg` for the alias, the `f` means filter.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [calcite] cshuo commented on issue #1741: [CALCITE-3721] Filter of distinct aggregate call is lost after applyi…

Posted by GitBox <gi...@apache.org>.
cshuo commented on issue #1741: [CALCITE-3721] Filter of distinct aggregate call is lost after applyi…
URL: https://github.com/apache/calcite/pull/1741#issuecomment-574137157
 
 
   See [pull/1758](https://github.com/apache/calcite/pull/1758)

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [calcite] cshuo closed pull request #1741: [CALCITE-3721] Filter of distinct aggregate call is lost after applyi…

Posted by GitBox <gi...@apache.org>.
cshuo closed pull request #1741: [CALCITE-3721] Filter of distinct aggregate call is lost after applyi…
URL: https://github.com/apache/calcite/pull/1741
 
 
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [calcite] cshuo commented on a change in pull request #1741: [CALCITE-3721] Filter of distinct aggregate call is lost after applyi…

Posted by GitBox <gi...@apache.org>.
cshuo commented on a change in pull request #1741: [CALCITE-3721] Filter of distinct aggregate call is lost after applyi…
URL: https://github.com/apache/calcite/pull/1741#discussion_r365494265
 
 

 ##########
 File path: core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
 ##########
 @@ -1474,6 +1474,27 @@ private void basePushFilterPastAggWithGroupingSets(boolean unchanged)
     sql(sql).with(program).check();
   }
 
+  @Test public void testDistinctWithFilterWithoutGroupBy() {
+    final String sql = "SELECT SUM(comm), COUNT(DISTINCT comm),\n"
+        + "COUNT(DISTINCT sal) FILTER (WHERE sal > 1000)\n"
+        + "FROM emp";
+    HepProgram program = new HepProgramBuilder()
+        .addRuleInstance(AggregateExpandDistinctAggregatesRule.INSTANCE)
+        .build();
+    sql(sql).with(program).check();
+  }
+
+  @Test public void testDistinctWithFilterAndGroupBy() {
+    final String sql = "SELECT deptno, SUM(comm), COUNT(DISTINCT comm),\n"
+        + "COUNT(DISTINCT sal) FILTER (WHERE sal > 1000)\n"
 
 Review comment:
   Hi, vlsi, the mininal reproducible case is there exists two aggregation call, one is distinct aggregate call with filter, the other can be distinct agg call or not. I've update the JIRA description with a comprehensible case. And the test cases here can be simplified a little actually.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [calcite] cshuo opened a new pull request #1741: [CALCITE-3721] Filter of distinct aggregate call is lost after applyi…

Posted by GitBox <gi...@apache.org>.
cshuo opened a new pull request #1741: [CALCITE-3721] Filter of distinct aggregate call is lost after applyi…
URL: https://github.com/apache/calcite/pull/1741
 
 
   …ng `AggregateExpandDistinctAggregatesRule`
   
   In `AggregateExpandDistinctAggregatesRule`, when the distinct aggregate call is rewritten using 
   groupingSets, the filter of the distinct aggregate call itself is lost unexpected.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [calcite] vlsi commented on a change in pull request #1741: [CALCITE-3721] Filter of distinct aggregate call is lost after applyi…

Posted by GitBox <gi...@apache.org>.
vlsi commented on a change in pull request #1741: [CALCITE-3721] Filter of distinct aggregate call is lost after applyi…
URL: https://github.com/apache/calcite/pull/1741#discussion_r365345915
 
 

 ##########
 File path: core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
 ##########
 @@ -1474,6 +1474,27 @@ private void basePushFilterPastAggWithGroupingSets(boolean unchanged)
     sql(sql).with(program).check();
   }
 
+  @Test public void testDistinctWithFilterWithoutGroupBy() {
+    final String sql = "SELECT SUM(comm), COUNT(DISTINCT comm),\n"
+        + "COUNT(DISTINCT sal) FILTER (WHERE sal > 1000)\n"
+        + "FROM emp";
+    HepProgram program = new HepProgramBuilder()
+        .addRuleInstance(AggregateExpandDistinctAggregatesRule.INSTANCE)
+        .build();
+    sql(sql).with(program).check();
+  }
+
+  @Test public void testDistinctWithFilterAndGroupBy() {
+    final String sql = "SELECT deptno, SUM(comm), COUNT(DISTINCT comm),\n"
+        + "COUNT(DISTINCT sal) FILTER (WHERE sal > 1000)\n"
 
 Review comment:
   Is this a minimal reproducible example?
   
   Frankly speaking, it is really hard to follow the code, and it is sad it is the most trivial example in the PR.
   
   I would prefer to have a simple example that makes it clear what the rule produces. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [calcite] danny0405 commented on a change in pull request #1741: [CALCITE-3721] Filter of distinct aggregate call is lost after applyi…

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #1741: [CALCITE-3721] Filter of distinct aggregate call is lost after applyi…
URL: https://github.com/apache/calcite/pull/1741#discussion_r366252358
 
 

 ##########
 File path: core/src/main/java/org/apache/calcite/rel/rules/AggregateExpandDistinctAggregatesRule.java
 ##########
 @@ -470,21 +492,20 @@ private void rewriteUsingGroupingSets(RelOptRuleCall call,
       if (!aggCall.isDistinct()) {
         aggregation = SqlStdOperatorTable.MIN;
         newArgList = ImmutableIntList.of(x++);
-        newFilterArg = filters.get(aggregate.getGroupSet());
+        newFilterArg = filters.get(Pair.of(aggregate.getGroupSet(), -1));
       } else {
         aggregation = aggCall.getAggregation();
         newArgList = remap(fullGroupSet, aggCall.getArgList());
-        newFilterArg =
-            filters.get(
-                ImmutableBitSet.of(aggCall.getArgList())
-                    .setIf(aggCall.filterArg, aggCall.filterArg >= 0)
-                    .union(aggregate.getGroupSet()));
+        final ImmutableBitSet groupSet = ImmutableBitSet.of(aggCall.getArgList())
+            .setIf(aggCall.filterArg, aggCall.filterArg >= 0)
 
 Review comment:
   Maybe we can reuse the group set for each agg call.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [calcite] danny0405 commented on a change in pull request #1741: [CALCITE-3721] Filter of distinct aggregate call is lost after applyi…

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #1741: [CALCITE-3721] Filter of distinct aggregate call is lost after applyi…
URL: https://github.com/apache/calcite/pull/1741#discussion_r366250026
 
 

 ##########
 File path: core/src/main/java/org/apache/calcite/rel/rules/AggregateExpandDistinctAggregatesRule.java
 ##########
 @@ -402,15 +400,27 @@ private void rewriteUsingGroupingSets(RelOptRuleCall call,
       Aggregate aggregate) {
     final Set<ImmutableBitSet> groupSetTreeSet =
         new TreeSet<>(ImmutableBitSet.ORDERING);
+    // groupSet to distinct filter arg map,
+    // filterArg will be -1 for groupSet of non-distinct agg.
+    final Map<ImmutableBitSet, Set<Integer>> distinctFilterArgMap = new HashMap<>();
     for (AggregateCall aggCall : aggregate.getAggCallList()) {
+      ImmutableBitSet groupSet;
+      int filterArg;
       if (!aggCall.isDistinct()) {
+        filterArg = -1;
+        groupSet = aggregate.getGroupSet();
         groupSetTreeSet.add(aggregate.getGroupSet());
       } else {
-        groupSetTreeSet.add(
+        filterArg = aggCall.filterArg;
+        groupSet =
             ImmutableBitSet.of(aggCall.getArgList())
-                .setIf(aggCall.filterArg, aggCall.filterArg >= 0)
-                .union(aggregate.getGroupSet()));
+                .setIf(filterArg, filterArg >= 0)
+                .union(aggregate.getGroupSet());
+        groupSetTreeSet.add(groupSet);
       }
+      Set<Integer> filterList = distinctFilterArgMap
+          .computeIfAbsent(groupSet, g -> new HashSet<>());
+      filterList.add(filterArg);
 
 Review comment:
   Add document to explain why we need a `Set` to keep the filter arg.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [calcite] danny0405 commented on a change in pull request #1741: [CALCITE-3721] Filter of distinct aggregate call is lost after applyi…

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #1741: [CALCITE-3721] Filter of distinct aggregate call is lost after applyi…
URL: https://github.com/apache/calcite/pull/1741#discussion_r366250355
 
 

 ##########
 File path: core/src/main/java/org/apache/calcite/rel/rules/AggregateExpandDistinctAggregatesRule.java
 ##########
 @@ -432,31 +442,43 @@ private void rewriteUsingGroupingSets(RelOptRuleCall call,
     relBuilder.push(aggregate.getInput());
     final int groupCount = fullGroupSet.cardinality();
 
-    final Map<ImmutableBitSet, Integer> filters = new LinkedHashMap<>();
-    final int z = groupCount + distinctAggCalls.size();
+    // get the base ordinal for filter args for different groupSet.
+    final Map<Pair<ImmutableBitSet, Integer>, Integer> filters = new LinkedHashMap<>();
+    int z = groupCount + distinctAggCalls.size();
+    for (ImmutableBitSet groupSet: groupSets) {
+      Set<Integer> filterArgList = distinctFilterArgMap.get(groupSet);
+      for (Integer filterArg: filterArgList) {
+        filters.put(Pair.of(groupSet, filterArg), z);
+        z += 1;
+      }
+    }
+
     distinctAggCalls.add(
         AggregateCall.create(SqlStdOperatorTable.GROUPING, false, false, false,
             ImmutableIntList.copyOf(fullGroupSet), -1, RelCollations.EMPTY,
             groupSets.size(), relBuilder.peek(), null, "$g"));
-    for (Ord<ImmutableBitSet> groupSet : Ord.zip(groupSets)) {
-      filters.put(groupSet.e, z + groupSet.i);
-    }
-
     relBuilder.aggregate(relBuilder.groupKey(fullGroupSet, groupSets),
         distinctAggCalls);
-    final RelNode distinct = relBuilder.peek();
 
     // GROUPING returns an integer (0 or 1). Add a project to convert those
     // values to BOOLEAN.
     if (!filters.isEmpty()) {
       final List<RexNode> nodes = new ArrayList<>(relBuilder.fields());
       final RexNode nodeZ = nodes.remove(nodes.size() - 1);
-      for (Map.Entry<ImmutableBitSet, Integer> entry : filters.entrySet()) {
-        final long v = groupValue(fullGroupSet, entry.getKey());
+      for (Map.Entry<Pair<ImmutableBitSet, Integer>, Integer> entry : filters.entrySet()) {
+        final long v = groupValue(fullGroupSet, entry.getKey().left);
+        int distinctFilterArg = remap(fullGroupSet, entry.getKey().right);
+        RexNode expr = relBuilder.equals(nodeZ, relBuilder.literal(v));
+        if (distinctFilterArg > -1) {
+          // merge the filter of the distinct aggregate call itself.
+          RexBuilder rexBuilder = aggregate.getCluster().getRexBuilder();
+          expr = relBuilder.and(expr,
+              rexBuilder.makeCall(SqlStdOperatorTable.IS_TRUE,
+                  relBuilder.field(distinctFilterArg)));
 
 Review comment:
   Use `RelBuilder.call` instead of `RexBuilder.makeCall`.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [calcite] cshuo opened a new pull request #1741: [CALCITE-3721] Filter of distinct aggregate call is lost after applyi…

Posted by GitBox <gi...@apache.org>.
cshuo opened a new pull request #1741: [CALCITE-3721] Filter of distinct aggregate call is lost after applyi…
URL: https://github.com/apache/calcite/pull/1741
 
 
   …ng `AggregateExpandDistinctAggregatesRule`
   
   In `AggregateExpandDistinctAggregatesRule`, when the distinct aggregate call is rewritten using 
   groupingSets, the filter of the distinct aggregate call itself is lost unexpected.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [calcite] cshuo closed pull request #1741: [CALCITE-3721] Filter of distinct aggregate call is lost after applyi…

Posted by GitBox <gi...@apache.org>.
cshuo closed pull request #1741: [CALCITE-3721] Filter of distinct aggregate call is lost after applyi…
URL: https://github.com/apache/calcite/pull/1741
 
 
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [calcite] cshuo closed pull request #1741: [CALCITE-3721] Filter of distinct aggregate call is lost after applyi…

Posted by GitBox <gi...@apache.org>.
cshuo closed pull request #1741: [CALCITE-3721] Filter of distinct aggregate call is lost after applyi…
URL: https://github.com/apache/calcite/pull/1741
 
 
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services