beyond1920 commented on a change in pull request #15412:
URL: https://github.com/apache/flink/pull/15412#discussion_r603271298



##########
File path: 
flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/WindowUtil.scala
##########
@@ -198,6 +204,79 @@ object WindowUtil {
     new TimeAttributeWindowingStrategy(windowSpec, timeAttributeType, 
timeIndex)
   }
 
+  /**
+   * Window TVF based aggregations don't support early-fire and late-fire,
+   * throws exception when the configurations are set.
+   */
+  def checkEmitConfiguration(tableConfig: TableConfig): Unit = {
+    val conf = tableConfig.getConfiguration
+    if (conf.getBoolean(TABLE_EXEC_EMIT_EARLY_FIRE_ENABLED) ||
+      conf.getBoolean(TABLE_EXEC_EMIT_LATE_FIRE_ENABLED)) {
+      throw new TableException("Currently, window table function based 
aggregate doesn't " +
+        s"support early-fire and late-fire configuration " +
+        s"'${TABLE_EXEC_EMIT_EARLY_FIRE_ENABLED.key()}' and " +
+        s"'${TABLE_EXEC_EMIT_LATE_FIRE_ENABLED.key()}'.")
+    }
+  }
+
+  // 
------------------------------------------------------------------------------------------
+  // RelNode RowType
+  // 
------------------------------------------------------------------------------------------
+
+  def deriveWindowAggregateRowType(

Review comment:
       It's better to update `deriveRowType` method in 
`StreamPhysicalWindowAggregate` to use 
`WindowUtil`#deriveWindowAggregateRowType?

##########
File path: 
flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/physical/stream/TwoStageOptimizedWindowAggregateRule.java
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.physical.stream;
+
+import org.apache.flink.table.api.TableConfig;
+import 
org.apache.flink.table.planner.plan.logical.SliceAttachedWindowingStrategy;
+import org.apache.flink.table.planner.plan.logical.WindowingStrategy;
+import org.apache.flink.table.planner.plan.nodes.FlinkConventions;
+import 
org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExchange;
+import 
org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalGlobalWindowAggregate;
+import 
org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLocalWindowAggregate;
+import 
org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowAggregate;
+import 
org.apache.flink.table.planner.plan.rules.physical.FlinkExpandConversionRule;
+import org.apache.flink.table.planner.plan.trait.FlinkRelDistribution;
+import org.apache.flink.table.planner.plan.trait.FlinkRelDistributionTraitDef;
+import org.apache.flink.table.planner.plan.trait.ModifyKindSetTrait;
+import org.apache.flink.table.planner.plan.trait.UpdateKindTrait;
+import org.apache.flink.table.planner.plan.utils.AggregateUtil;
+import org.apache.flink.table.planner.utils.AggregatePhaseStrategy;
+
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+
+import java.util.stream.IntStream;
+
+import static org.apache.flink.table.planner.utils.ShortcutUtils.unwrapContext;
+import static 
org.apache.flink.table.planner.utils.TableConfigUtils.getAggPhaseStrategy;
+
+/**
+ * Rule that matches {@link StreamPhysicalWindowAggregate} on {@link 
StreamPhysicalExchange} with
+ * following condition:
+ *
+ * <ul>
+ *   <li>the applied windowing is not on processing-time, because 
processing-time should be
+ *       materialized in a single node.
+ *   <li>two-phase aggregation is enabled in given TableConfig.
+ *   <li>all aggregate functions support merge() method.
+ *   <li>the input of exchange does not satisfy the shuffle distribution
+ * </ul>
+ *
+ * <p>This optimization is as known as local-global optimization for reducing 
data-shuffling. It
+ * splits window aggregation into two-stage window aggregation, i.e. 
local-aggregation and
+ * global-aggregation. The local-aggregation produces a partial result for 
each group and window
+ * slice before shuffle in stage 1, and then the partially aggregated results 
are shuffled by group
+ * key to global-aggregation which produces the final result in stage 2.
+ */
+public class TwoStageOptimizedWindowAggregateRule extends RelOptRule {
+
+    public static final TwoStageOptimizedWindowAggregateRule INSTANCE =
+            new TwoStageOptimizedWindowAggregateRule();
+
+    private TwoStageOptimizedWindowAggregateRule() {
+        super(
+                operand(
+                        StreamPhysicalWindowAggregate.class,
+                        operand(StreamPhysicalExchange.class, 
operand(RelNode.class, any()))),
+                "TwoStageOptimizedWindowAggregateRule");
+    }
+
+    @Override
+    public boolean matches(RelOptRuleCall call) {
+        final StreamPhysicalWindowAggregate windowAgg = call.rel(0);
+        final RelNode realInput = call.rel(2);
+        final TableConfig tableConfig = 
unwrapContext(call.getPlanner()).getTableConfig();
+        final WindowingStrategy windowing = windowAgg.windowing();
+
+        // the two-phase optimization must be enabled
+        if (getAggPhaseStrategy(tableConfig) == 
AggregatePhaseStrategy.ONE_PHASE) {
+            return false;
+        }
+
+        // processing time window doesn't support two-phase,
+        // otherwise the processing-time can't be materialized in a single node
+        if (!windowing.isRowtime()) {
+            return false;
+        }
+
+        // all aggregate function should support merge() method
+        if 
(!AggregateUtil.doAllSupportPartialMerge(windowAgg.aggInfoList().aggInfos())) {
+            return false;
+        }
+
+        return !isInputSatisfyRequiredDistribution(realInput, 
windowAgg.grouping());
+    }
+
+    @Override
+    public void onMatch(RelOptRuleCall call) {
+        final StreamPhysicalWindowAggregate windowAgg = call.rel(0);
+        final RelNode realInput = call.rel(2);
+
+        RelTraitSet localTraitSet =
+                realInput
+                        .getTraitSet()
+                        .plus(ModifyKindSetTrait.INSERT_ONLY())
+                        .plus(UpdateKindTrait.NONE());
+        StreamPhysicalLocalWindowAggregate localAgg =
+                new StreamPhysicalLocalWindowAggregate(
+                        windowAgg.getCluster(),
+                        localTraitSet,
+                        realInput,
+                        windowAgg.grouping(),
+                        windowAgg.aggCalls(),
+                        windowAgg.windowing());
+
+        // grouping keys is forwarded by local agg, use indices instead of 
groupings
+        int[] globalGrouping = IntStream.range(0, 
windowAgg.grouping().length).toArray();
+        FlinkRelDistribution globalDistribution = 
createDistribution(globalGrouping);
+        // create exchange if needed
+        RelNode newInput =
+                FlinkExpandConversionRule.satisfyDistribution(
+                        FlinkConventions.STREAM_PHYSICAL(), localAgg, 
globalDistribution);
+        RelTraitSet globalAggProvidedTraitSet = windowAgg.getTraitSet();
+        SliceAttachedWindowingStrategy globalWindowing =

Review comment:
       It's better to distinguish two different cases here:
   1. If `windowStrategy` of original `StreamPhysicalWindowAggregate` is 
`TimeAttributeWindowingStrategy`,
   then `WindowingStrategy` of equivalent `StreamPhysicalGlobalWindowAggregate` 
is `SliceAttachedWindowingStrategy` like the current code.
   2. If `WindowingStrategy` of original `StreamPhysicalWindowAggregate` is 
`WindowAttachedWindowingStrategy`, then `WindowingStrategy` of equivalent 
`StreamPhysicalGlobalWindowAggregate` could still be 
`WindowAttachedWindowingStrategy`. Besides, there is no need to add `sliceEnd` 
in `StreamPhysicalLocalWindowAggregate`.
   

##########
File path: 
flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/RelExplainUtil.scala
##########
@@ -261,57 +261,18 @@ object RelExplainUtil {
     }.mkString(", ")
   }
 
-  def streamGroupAggregationToString(
-      inputRowType: RelDataType,
-      outputRowType: RelDataType,
-      aggCalls: Seq[AggregateCall],
-      grouping: Array[Int]): String = {
-    val inputFieldNames = inputRowType.getFieldNames
-    val outputFieldNames = outputRowType.getFieldNames
-    val aggStrings = aggCalls.map { call =>
-      val distinct = if (call.isDistinct) {
-        if (call.getArgList.size() == 0) {
-          "DISTINCT"
-        } else {
-          "DISTINCT "
-        }
-      } else {
-        ""
-      }
-      val newArgList = call.getArgList.map(_.toInt).toList
-      val argListNames = if (newArgList.nonEmpty) {
-        newArgList.map(inputFieldNames(_)).mkString(", ")
-      } else {
-        "*"
-      }
-
-      if (call.filterArg >= 0 && call.filterArg < inputFieldNames.size) {
-        s"${call.getAggregation}($distinct$argListNames) FILTER " +
-          s"${inputFieldNames(call.filterArg)}"
-      } else {
-        s"${call.getAggregation}($distinct$argListNames)"
-      }
-    }
-    (grouping.map(inputFieldNames(_)) ++ aggStrings).zip(
-      grouping.indices.map(outputFieldNames(_)) ++ outputFieldNames).map {
-      case (f, o) => if (f == o) {
-        f
-      } else {
-        s"$f AS $o"
-      }
-    }.mkString(", ")
-  }
-
   def streamGroupAggregationToString(
       inputRowType: RelDataType,
       outputRowType: RelDataType,
       aggInfoList: AggregateInfoList,
       grouping: Array[Int],
       shuffleKey: Option[Array[Int]] = None,
+      windowProperties: Seq[PlannerNamedWindowProperty] = Seq(),

Review comment:
       It's better to split streamGroupAggregationToString into two different 
methods, one for window aggregate, one for  unbounded aggregate.




-- 
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:
[email protected]


Reply via email to