snuyanzin commented on code in PR #27383:
URL: https://github.com/apache/flink/pull/27383#discussion_r2678605005


##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/SplitAggregateRule.java:
##########
@@ -0,0 +1,616 @@
+/*
+ * 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.flink.table.planner.plan.rules.logical;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.table.api.TableConfig;
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.api.config.OptimizerConfigOptions;
+import org.apache.flink.table.planner.calcite.FlinkLogicalRelFactories;
+import org.apache.flink.table.planner.calcite.FlinkRelBuilder;
+import org.apache.flink.table.planner.functions.sql.FlinkSqlOperatorTable;
+import 
org.apache.flink.table.planner.functions.sql.SqlFirstLastValueAggFunction;
+import org.apache.flink.table.planner.plan.PartialFinalType;
+import org.apache.flink.table.planner.plan.logical.SessionWindowSpec;
+import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery;
+import org.apache.flink.table.planner.plan.nodes.FlinkRelNode;
+import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalAggregate;
+import org.apache.flink.table.planner.plan.trait.RelWindowProperties;
+import org.apache.flink.table.planner.plan.utils.AggregateUtil;
+import org.apache.flink.table.planner.plan.utils.ExpandUtil;
+import org.apache.flink.table.planner.plan.utils.WindowUtil;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelRule;
+import org.apache.calcite.rel.RelCollations;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.SqlAggFunction;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.fun.SqlMinMaxAggFunction;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.ImmutableIntList;
+import org.apache.commons.lang3.ArrayUtils;
+import org.immutables.value.Value;
+
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import scala.collection.JavaConverters;
+
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.AVG;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.COUNT;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.MAX;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.MIN;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.SINGLE_VALUE;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.SUM;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.SUM0;
+import static 
org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig;
+
+/**
+ * Planner rule that splits aggregations containing distinct aggregates, e.g, 
count distinct, into
+ * partial aggregations and final aggregations.
+ *
+ * <p>This rule rewrites an aggregate query with distinct aggregations into an 
expanded double
+ * aggregations. The first aggregation compute the results in sub-partition 
and the results are
+ * combined by the second aggregation.
+ *
+ * <pre>
+ * Examples:
+ *
+ * MyTable: a: BIGINT, b: INT, c: VARCHAR
+ *
+ * Original records:
+ * | a | b | c  |
+ * |:-:|:-:|:--:|
+ * | 1 | 1 | c1 |
+ * | 1 | 2 | c1 |
+ * | 2 | 1 | c2 |
+ *
+ * SQL: SELECT SUM(DISTINCT b), COUNT(DISTINCT c), AVG(b) FROM MyTable GROUP 
BY a
+ *
+ * flink logical plan:
+ * {@code
+ * FlinkLogicalCalc(select=[$f1 AS EXPR$0, $f2 AS EXPR$1, CAST(IF(=($f4, 
0:BIGINT), null:INTEGER,
+ *  /($f3, $f4))) AS EXPR$2])
+ * +- FlinkLogicalAggregate(group=[{0}], agg#0=[SUM($3)], agg#1=[$SUM0($4)], 
agg#2=[$SUM0($5)],
+ * agg#3=[$SUM0($6)])
+ *    +- FlinkLogicalAggregate(group=[{0, 3, 4}], agg#0=[SUM(DISTINCT $1) 
FILTER $5],
+ *    agg#1=[COUNT(DISTINCT $2) FILTER $6], agg#2=[$SUM0($1) FILTER $7],
+ *    agg#3=[COUNT($1) FILTER $7])
+ *       +- FlinkLogicalCalc(select=[a, b, c, $f3, $f4, =($e, 1) AS $g_1, 
=($e, 2) AS $g_2,
+ *       =($e, 3) AS $g_3])
+ *          +- FlinkLogicalExpand(projects=[a, b, c, $f3, $f4, $e])
+ *             +- FlinkLogicalCalc(select=[a, b, c, MOD(HASH_CODE(b), 1024) AS 
$f3,
+ *               MOD(HASH_CODE(c), 1024) AS $f4])
+ *                +- FlinkLogicalTableSourceScan(table=[[MyTable]], fields=[a, 
b, c])
+ * }
+ *
+ * '$e = 1' is equivalent to 'group by a, hash(b) % 1024' '$e = 2' is 
equivalent to 'group by a,
+ * hash(c) % 1024' '$e = 3' is equivalent to 'group by a
+ *
+ * Expanded records: 
\+-----+-----+-----+------------------+------------------+-----+ \| a | b | c |
+ * hash(b) % 1024 | hash(c) % 1024 | $e |
+ * \+-----+-----+-----+------------------+------------------+-----+ ---+--- \| 
1 | 1 | c1 | hash(b)
+ * % 1024 | null | 1 | | 
\+-----+-----+-----+------------------+------------------+-----+ | \| 1 | 1
+ * \| c1 | null | hash(c) % 1024 | 2 | records expanded by record1
+ * \+-----+-----+-----+------------------+-----------------+------+ | \| 1 | 1 
| c1 | null | null |
+ * 3 | | \+-----+-----+-----+------------------+-----------------+------+ 
---+--- \| 1 | 2 | c1 |
+ * hash(b) % 1024 | null | 1 | | 
\+-----+-----+-----+------------------+-----------------+------+ |
+ * \| 1 | 2 | c1 | null | hash(c) % 1024 | 2 | records expanded by record2
+ * \+-----+-----+-----+------------------+-----------------+------+ | \| 1 | 2 
| c1 | null | null |
+ * 3 | | \+-----+-----+-----+------------------+-----------------+------+ 
---+--- \| 2 | 1 | c2 |
+ * hash(b) % 1024 | null | 1 | | 
\+-----+-----+-----+------------------+-----------------+------+ |
+ * \| 2 | 1 | c2 | null | hash(c) % 1024 | 2 | records expanded by record3
+ * \+-----+-----+-----+------------------+-----------------+------+ | \| 2 | 1 
| c2 | null | null |
+ * 3 | | \+-----+-----+-----+------------------+-----------------+------+ 
---+---
+ * </pre>
+ *
+ * <p>NOTES: this rule is only used for Stream now.
+ */
[email protected]
+public class SplitAggregateRule extends 
RelRule<SplitAggregateRule.SplitAggregateRuleConfig> {
+    public static final SplitAggregateRule INSTANCE =
+            SplitAggregateRule.SplitAggregateRuleConfig.DEFAULT.toRule();
+
+    protected SplitAggregateRule(SplitAggregateRuleConfig config) {
+        super(config);
+    }
+
+    @Override
+    public boolean matches(RelOptRuleCall call) {
+        TableConfig tableConfig = unwrapTableConfig(call);
+        FlinkLogicalAggregate agg = call.rel(0);
+
+        boolean splitDistinctAggEnabled =
+                
tableConfig.get(OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED);
+        boolean isAllAggSplittable = 
AggregateUtil.doAllAggSupportSplit(agg.getAggCallList());
+
+        // disable distinct split for processing-time window,
+        // because the semantic is not clear to materialize processing-time 
window in two aggregates
+        FlinkRelMetadataQuery fmq = (FlinkRelMetadataQuery) 
call.getMetadataQuery();
+        RelWindowProperties windowProps = 
fmq.getRelWindowProperties(agg.getInput());
+        boolean isWindowAgg =
+                WindowUtil.groupingContainsWindowStartEnd(agg.getGroupSet(), 
windowProps);
+        boolean isProctimeWindowAgg = isWindowAgg && !windowProps.isRowtime();
+
+        // disable distinct split for session window,
+        // otherwise window assigner results may be different
+        boolean isSessionWindowAgg =
+                isWindowAgg && windowProps.getWindowSpec() instanceof 
SessionWindowSpec;
+        // TableAggregate is not supported. see also FLINK-21923.
+        boolean isTableAgg = 
AggregateUtil.isTableAggregate(agg.getAggCallList());
+
+        return agg.partialFinalType() == PartialFinalType.NONE
+                && agg.containsDistinctCall()
+                && splitDistinctAggEnabled
+                && isAllAggSplittable
+                && !isProctimeWindowAgg
+                && !isTableAgg
+                && !isSessionWindowAgg;
+    }
+
+    @Override
+    public void onMatch(RelOptRuleCall call) {
+        TableConfig tableConfig = unwrapTableConfig(call);
+        FlinkLogicalAggregate originalAggregate = call.rel(0);
+        List<AggregateCall> aggCalls = originalAggregate.getAggCallList();
+        FlinkRelNode input = call.rel(1);
+        RelOptCluster cluster = originalAggregate.getCluster();
+        FlinkRelBuilder relBuilder = (FlinkRelBuilder) call.builder();
+        relBuilder.push(input);
+        int[] aggGroupSet = originalAggregate.getGroupSet().toArray();
+
+        // STEP 1: add hash fields if necessary
+        int[] hashFieldIndexes =
+                IntStream.range(0, aggCalls.size())
+                        .filter(i -> 
SplitAggregateRule.needAddHashFields(aggCalls.get(i)))
+                        .flatMap(
+                                i ->
+                                        Arrays.stream(
+                                                
SplitAggregateRule.getArgIndexes(aggCalls.get(i))))
+                        .distinct()
+                        .filter(index -> !ArrayUtils.contains(aggGroupSet, 
index))
+                        .sorted()
+                        .toArray();
+
+        Map<Integer, Integer> hashFieldsMap = new HashMap<>();
+        int buckets =
+                tableConfig.get(
+                        
OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_BUCKET_NUM);
+
+        if (hashFieldIndexes.length > 0) {
+            List<RexNode> projects = new ArrayList<>(relBuilder.fields());
+            int hashFieldsOffset = projects.size();
+
+            IntStream.range(0, hashFieldIndexes.length)
+                    .forEach(
+                            index -> {
+                                int hashFieldIdx = hashFieldIndexes[index];
+                                RexNode hashField = 
relBuilder.field(hashFieldIdx);
+                                // hash(f) % buckets
+                                RexNode node =
+                                        relBuilder.call(
+                                                SqlStdOperatorTable.MOD,
+                                                relBuilder.call(
+                                                        
FlinkSqlOperatorTable.HASH_CODE, hashField),
+                                                relBuilder.literal(buckets));
+
+                                projects.add(node);
+                                hashFieldsMap.put(hashFieldIdx, 
hashFieldsOffset + index);
+                            });
+            relBuilder.project(projects);
+        }
+
+        // STEP 2: construct partial aggregates
+        Set<ImmutableBitSet> groupSetTreeSet = new 
TreeSet<>(ImmutableBitSet.ORDERING);
+        Map<AggregateCall, ImmutableBitSet> aggInfoToGroupSetMap = new 
HashMap<>();
+        int newGroupSetsNum = 0;
+        for (AggregateCall aggCall : aggCalls) {
+            ImmutableBitSet groupSet;
+            if (SplitAggregateRule.needAddHashFields(aggCall)) {
+                List<Integer> newIndexes =
+                        
Arrays.stream(SplitAggregateRule.getArgIndexes(aggCall))
+                                .mapToObj(
+                                        argIndex -> 
hashFieldsMap.getOrDefault(argIndex, argIndex))
+                                .collect(Collectors.toList());
+                groupSet = 
ImmutableBitSet.of(newIndexes).union(ImmutableBitSet.of(aggGroupSet));
+                // Only increment groupSet number if aggregate call needs add 
new different hash
+                // fields
+                // e.g SQL1: SELECT COUNT(DISTINCT a), MAX(a) FROM T group by b
+                // newGroupSetsNum is 1 because two agg function add same hash 
field
+                // e.g SQL2: SELECT COUNT(DISTINCT a), COUNT(b) FROM T group 
by c
+                // newGroupSetsNum is 1 because only COUNT(DISTINCT a) adds a 
new hash field
+                // e.g SQL3: SELECT COUNT(DISTINCT a), COUNT(DISTINCT b) FROM 
T group by b
+                // newGroupSetsNum is 2 because COUNT(DISTINCT a), 
COUNT(DISTINCT b) both add hash
+                // field
+                if (!groupSetTreeSet.contains(groupSet)) {
+                    newGroupSetsNum += 1;
+                }
+            } else {
+                groupSet = ImmutableBitSet.of(aggGroupSet);
+            }
+            groupSetTreeSet.add(groupSet);
+            aggInfoToGroupSetMap.put(aggCall, groupSet);
+        }
+        ImmutableList<ImmutableBitSet> groupSets = 
ImmutableList.copyOf(groupSetTreeSet);
+        ImmutableBitSet fullGroupSet = ImmutableBitSet.union(groupSets);
+
+        // STEP 2.1: expand input fields
+        List<AggregateCall> partialAggCalls = new ArrayList<>();
+        Map<AggregateCall, ImmutableBitSet> partialAggCallToGroupSetMap = new 
HashMap<>();
+        aggCalls.stream()

Review Comment:
   since we're doing `foreach` only, do we need `stream` ?



-- 
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.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to