You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by zs...@apache.org on 2022/12/28 08:41:02 UTC

[ignite-3] branch main updated: IGNITE-18206 Sql. Prohibit using 2-phase aggregates with distinct accumulator on map phase - Fixes #1456.

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

zstan pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/ignite-3.git


The following commit(s) were added to refs/heads/main by this push:
     new dc800016ea IGNITE-18206 Sql. Prohibit using 2-phase aggregates with distinct accumulator on map phase - Fixes #1456.
dc800016ea is described below

commit dc800016ea669e580a2182c5ea84a27e926a7d87
Author: zstan <st...@gmail.com>
AuthorDate: Wed Dec 28 11:38:20 2022 +0300

    IGNITE-18206 Sql. Prohibit using 2-phase aggregates with distinct accumulator on map phase - Fixes #1456.
    
    Signed-off-by: zstan <st...@gmail.com>
---
 .../sql/engine/rel/set/IgniteMapSetOp.java         | 10 +--
 .../engine/rule/HashAggregateConverterRule.java    |  8 ++-
 .../engine/rule/SortAggregateConverterRule.java    |  7 +-
 .../ignite/internal/sql/engine/util/PlanUtils.java | 47 ++++++++++++++
 .../sql/engine/planner/AggregatePlannerTest.java   | 74 +++++++++++++++++++++-
 5 files changed, 134 insertions(+), 12 deletions(-)

diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/rel/set/IgniteMapSetOp.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/rel/set/IgniteMapSetOp.java
index a3f52b812d..6b1a8fa9f3 100644
--- a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/rel/set/IgniteMapSetOp.java
+++ b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/rel/set/IgniteMapSetOp.java
@@ -40,7 +40,7 @@ import org.apache.ignite.internal.sql.engine.util.Commons;
 public interface IgniteMapSetOp extends IgniteSetOp {
     /** {@inheritDoc} */
     @Override
-    public default List<Pair<RelTraitSet, List<RelTraitSet>>> deriveRewindability(
+    default List<Pair<RelTraitSet, List<RelTraitSet>>> deriveRewindability(
             RelTraitSet nodeTraits,
             List<RelTraitSet> inputTraits
     ) {
@@ -58,7 +58,7 @@ public interface IgniteMapSetOp extends IgniteSetOp {
 
     /** {@inheritDoc} */
     @Override
-    public default List<Pair<RelTraitSet, List<RelTraitSet>>> deriveDistribution(
+    default List<Pair<RelTraitSet, List<RelTraitSet>>> deriveDistribution(
             RelTraitSet nodeTraits,
             List<RelTraitSet> inputTraits
     ) {
@@ -77,7 +77,7 @@ public interface IgniteMapSetOp extends IgniteSetOp {
 
     /** {@inheritDoc} */
     @Override
-    public default List<Pair<RelTraitSet, List<RelTraitSet>>> deriveCorrelation(
+    default List<Pair<RelTraitSet, List<RelTraitSet>>> deriveCorrelation(
             RelTraitSet nodeTraits,
             List<RelTraitSet> inTraits
     ) {
@@ -91,7 +91,7 @@ public interface IgniteMapSetOp extends IgniteSetOp {
     }
 
     /** Build RowType for MAP node. */
-    public default RelDataType buildRowType() {
+    default RelDataType buildRowType() {
         RelDataTypeFactory typeFactory = Commons.typeFactory(getCluster());
 
         assert typeFactory instanceof IgniteTypeFactory;
@@ -106,7 +106,7 @@ public interface IgniteMapSetOp extends IgniteSetOp {
 
     /** {@inheritDoc} */
     @Override
-    public default AggregateType aggregateType() {
+    default AggregateType aggregateType() {
         return AggregateType.MAP;
     }
 }
diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/rule/HashAggregateConverterRule.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/rule/HashAggregateConverterRule.java
index 192e2edd31..bf0445f2a5 100644
--- a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/rule/HashAggregateConverterRule.java
+++ b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/rule/HashAggregateConverterRule.java
@@ -17,6 +17,8 @@
 
 package org.apache.ignite.internal.sql.engine.rule;
 
+import static org.apache.ignite.internal.sql.engine.util.PlanUtils.complexDistinctAgg;
+
 import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelOptPlanner;
 import org.apache.calcite.plan.RelOptRule;
@@ -33,8 +35,8 @@ import org.apache.ignite.internal.sql.engine.trait.IgniteDistributions;
 import org.apache.ignite.internal.sql.engine.util.HintUtils;
 
 /**
- * HashAggregateConverterRule.
- * TODO Documentation https://issues.apache.org/jira/browse/IGNITE-15859
+ * Planner rule that recognizes a {@link org.apache.calcite.rel.core.Aggregate}
+ * and in relation to distribution and additional conditions produce appropriate node.
  */
 public class HashAggregateConverterRule {
     public static final RelOptRule COLOCATED = new ColocatedHashAggregateConverterRule();
@@ -83,7 +85,7 @@ public class HashAggregateConverterRule {
         @Override
         protected PhysicalNode convert(RelOptPlanner planner, RelMetadataQuery mq,
                 LogicalAggregate agg) {
-            if (HintUtils.isExpandDistinctAggregate(agg)) {
+            if (complexDistinctAgg(agg.getAggCallList()) || HintUtils.isExpandDistinctAggregate(agg)) {
                 return null;
             }
 
diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/rule/SortAggregateConverterRule.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/rule/SortAggregateConverterRule.java
index e227143f35..e4ee216647 100644
--- a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/rule/SortAggregateConverterRule.java
+++ b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/rule/SortAggregateConverterRule.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.sql.engine.rule;
 
+import static org.apache.ignite.internal.sql.engine.util.PlanUtils.complexDistinctAgg;
 import static org.apache.ignite.internal.util.CollectionUtils.nullOrEmpty;
 
 import org.apache.calcite.plan.RelOptCluster;
@@ -37,8 +38,8 @@ import org.apache.ignite.internal.sql.engine.trait.TraitUtils;
 import org.apache.ignite.internal.sql.engine.util.HintUtils;
 
 /**
- * SortAggregateConverterRule.
- * TODO Documentation https://issues.apache.org/jira/browse/IGNITE-15859
+ * Planner rule that recognizes a {@link org.apache.calcite.rel.core.Aggregate}
+ * and in relation to distribution and additional conditions produce appropriate node, require sorted input.
  */
 public class SortAggregateConverterRule {
     public static final RelOptRule COLOCATED = new ColocatedSortAggregateConverterRule();
@@ -105,7 +106,7 @@ public class SortAggregateConverterRule {
                 return null;
             }
 
-            if (HintUtils.isExpandDistinctAggregate(agg)) {
+            if (complexDistinctAgg(agg.getAggCallList()) || HintUtils.isExpandDistinctAggregate(agg)) {
                 return null;
             }
 
diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/util/PlanUtils.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/util/PlanUtils.java
new file mode 100644
index 0000000000..9a88fe7bd4
--- /dev/null
+++ b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/util/PlanUtils.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.sql.engine.util;
+
+import java.util.List;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.sql.fun.SqlAvgAggFunction;
+import org.apache.calcite.sql.fun.SqlCountAggFunction;
+import org.apache.calcite.sql.fun.SqlSumAggFunction;
+
+/**
+ * Plan util methods.
+ */
+public class PlanUtils {
+    /**
+     * Return {@code true} if observes AGGREGATE and DISTINCT simultaneously.
+     *
+     * @param aggCalls Aggregates.
+     * @return {@code true} If found, {@code false} otherwise.
+     */
+    public static boolean complexDistinctAgg(List<AggregateCall> aggCalls) {
+        for (AggregateCall call : aggCalls) {
+            if (call.isDistinct()
+                    && (call.getAggregation() instanceof SqlCountAggFunction
+                    || call.getAggregation() instanceof SqlAvgAggFunction
+                    || call.getAggregation() instanceof SqlSumAggFunction)) {
+                return true;
+            }
+        }
+        return false;
+    }
+}
diff --git a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/AggregatePlannerTest.java b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/AggregatePlannerTest.java
index a281edf709..2faa70a47d 100644
--- a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/AggregatePlannerTest.java
+++ b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/AggregatePlannerTest.java
@@ -26,9 +26,11 @@ import static org.junit.jupiter.api.Assertions.assertNotNull;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.math.BigDecimal;
+import java.util.List;
 import java.util.UUID;
 import java.util.stream.Stream;
 import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.SingleRel;
 import org.apache.calcite.rel.core.Aggregate;
 import org.apache.calcite.rel.core.AggregateCall;
@@ -54,7 +56,9 @@ import org.apache.ignite.internal.sql.engine.schema.IgniteSchema;
 import org.apache.ignite.internal.sql.engine.trait.IgniteDistribution;
 import org.apache.ignite.internal.sql.engine.trait.IgniteDistributions;
 import org.apache.ignite.internal.sql.engine.trait.TraitUtils;
+import org.apache.ignite.internal.util.Pair;
 import org.hamcrest.core.IsInstanceOf;
+import org.junit.jupiter.api.Test;
 import org.junit.jupiter.params.ParameterizedTest;
 import org.junit.jupiter.params.provider.Arguments;
 import org.junit.jupiter.params.provider.EnumSource;
@@ -262,7 +266,7 @@ public class AggregatePlannerTest extends AbstractAggregatePlannerTest {
                 "Invalid plan\n" + RelOptUtil.toString(phys, SqlExplainLevel.ALL_ATTRIBUTES)
         );
 
-        // Check the first aggrgation step is SELECT DISTINCT (doesn't contains any accumulators)
+        // Check the first aggregation step is SELECT DISTINCT (doesn't contain any accumulators)
         assertTrue(
                 findNodes(phys, byClass(algo.reduce)).stream()
                         .allMatch(n -> ((IgniteReduceAggregateBase) n).getAggregateCalls().isEmpty()),
@@ -372,6 +376,74 @@ public class AggregatePlannerTest extends AbstractAggregatePlannerTest {
                 algo.rulesToDisable);
     }
 
+    /**
+     * Check that map aggregate does not contain distinct accumulator.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    public void mapAggregateWithoutDistinctAcc() throws Exception {
+        TestTable tbl = createAffinityTable("TEST");
+
+        IgniteSchema publicSchema = new IgniteSchema("PUBLIC");
+
+        publicSchema.addTable(tbl);
+
+        // TODO: https://issues.apache.org/jira/browse/IGNITE-18464 Colocated sort aggregates need to compose a plans with additional sort
+        //checkDistinctInMapAggNode("SELECT COUNT(DISTINCT val0) FROM test", publicSchema);
+        //checkDistinctInMapAggNode("SELECT AVG(DISTINCT val0) FROM test", publicSchema);
+        //checkDistinctInMapAggNode("SELECT SUM(DISTINCT val0) FROM test", publicSchema);
+        //checkDistinctInMapAggNode("SELECT MIN(DISTINCT val0) FROM test", publicSchema);
+        //checkDistinctInMapAggNode("SELECT MAX(DISTINCT val0) FROM test", publicSchema);
+
+        checkDistinctInMapAggNode("SELECT COUNT(DISTINCT val0) FROM test GROUP BY val1", publicSchema);
+
+        checkDistinctInMapAggNode("SELECT val1, COUNT(DISTINCT val0) as v1 FROM test GROUP BY val1", publicSchema);
+
+        checkDistinctInMapAggNode("SELECT AVG(DISTINCT val0) FROM test GROUP BY val1", publicSchema);
+
+        checkDistinctInMapAggNode("SELECT SUM(DISTINCT val0) FROM test GROUP BY val1", publicSchema);
+
+        checkDistinctInMapAggNode("SELECT MIN(DISTINCT val0) FROM test GROUP BY val1", publicSchema);
+
+        checkDistinctInMapAggNode("SELECT MAX(DISTINCT val0) FROM test GROUP BY val1", publicSchema);
+
+        checkDistinctInMapAggNode("SELECT val0 FROM test WHERE VAL1 = ANY(SELECT DISTINCT val1 FROM test)", publicSchema);
+    }
+
+    /**
+     * Check that plan does not contain distinct accumulators on map nodes.
+     *
+     * @param sql Request string.
+     * @param publicSchema Schema.
+     * @throws Exception If failed.
+     */
+    private void checkDistinctInMapAggNode(String sql, IgniteSchema publicSchema) throws Exception {
+        List<Pair<String[], Boolean>> disabledRules = List.of(new Pair<>(new String[]{""}, false),
+                new Pair<>(new String[]{"ColocatedHashAggregateConverterRule", "ColocatedSortAggregateConverterRule"}, true),
+                new Pair<>(new String[]{"ColocatedHashAggregateConverterRule", "ColocatedSortAggregateConverterRule",
+                        "MapReduceSortAggregateConverterRule"}, true),
+                new Pair<>(new String[]{"ColocatedHashAggregateConverterRule", "ColocatedSortAggregateConverterRule",
+                        "MapReduceHashAggregateConverterRule"}, true)
+        );
+
+        for (Pair<String[], Boolean> rules : disabledRules) {
+            IgniteRel phys = physicalPlan(sql, publicSchema, rules.getFirst());
+
+            List<RelNode> nodes = findNodes(phys, byClass(IgniteMapAggregateBase.class));
+
+            if (rules.getSecond()) {
+                assertThat("No mapper found", !nodes.isEmpty());
+            }
+
+            assertFalse(findNodes(phys, byClass(IgniteMapAggregateBase.class)).stream()
+                            .anyMatch(n -> ((Aggregate) n).getAggCallList().stream()
+                                    .anyMatch(AggregateCall::isDistinct)
+                            ),
+                    "Invalid plan\n" + RelOptUtil.toString(phys, SqlExplainLevel.ALL_ATTRIBUTES));
+        }
+    }
+
     private static Stream<Arguments> provideAlgoAndDistribution() {
         return Stream.of(
                 Arguments.of(AggregateAlgorithm.SORT, IgniteDistributions.broadcast()),