flink-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From GitBox <...@apache.org>
Subject [GitHub] [flink] godfreyhe commented on a change in pull request #16025: [FLINK-22714][table-planner-blink] Simplify window TVF to a simple window assigner if successor node is WindowRank or WindowJoin.
Date Tue, 29 Jun 2021 05:30:39 GMT

godfreyhe commented on a change in pull request #16025:
URL: https://github.com/apache/flink/pull/16025#discussion_r660269928



##########
File path: flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/physical/stream/SimplifyWindowTableFunctionRule.java
##########
@@ -0,0 +1,355 @@
+/*
+ * 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.TableException;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCalc;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExchange;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowTableFunction;
+
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.RelRule;
+import org.apache.calcite.rel.RelNode;
+
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * Planner rule that tries to simplify emit behavior of {@link StreamPhysicalWindowTableFunction}
to
+ * emit per record instead of emit after watermark passed window end if the successor node
is {@link
+ * StreamPhysicalWindowRank} or {@link StreamPhysicalWindowJoin} which with
+ * rowtime-WindowingStrategy.
+ */
+public class SimplifyWindowTableFunctionRule
+        extends RelRule<SimplifyWindowTableFunctionRule.Config> {
+
+    public static final RelOptRule WITH_WINDOW_RANK =
+            Config.EMPTY
+                    .withDescription("SimplifyWindowTableFunctionRuleWithWindowRank")
+                    .as(Config.class)
+                    .withWindowRank()
+                    .toRule();
+
+    public static final RelOptRule WITH_CALC_WINDOW_RANK =
+            Config.EMPTY
+                    .withDescription("SimplifyWindowTableFunctionRuleWithCalcWindowRank")
+                    .as(Config.class)
+                    .withCalcWindowRank()
+                    .toRule();
+
+    public static final RelOptRule WITH_WINDOW_JOIN =
+            Config.EMPTY
+                    .withDescription("SimplifyWindowTableFunctionRuleWithWindowJoin")
+                    .as(Config.class)
+                    .withWindowJoin()
+                    .toRule();
+
+    public static final RelOptRule WITH_LEFT_CALC_WINDOW_JOIN =
+            Config.EMPTY
+                    .withDescription("SimplifyWindowTableFunctionRuleWithLeftCalcWindowJoin")
+                    .as(Config.class)
+                    .withLeftCalcWindowJoin()
+                    .toRule();
+
+    public static final RelOptRule WITH_RIGHT_CALC_WINDOW_JOIN =
+            Config.EMPTY
+                    .withDescription("SimplifyWindowTableFunctionRuleWithRightCalcWindowJoin")
+                    .as(Config.class)
+                    .withRightCalcWindowJoin()
+                    .toRule();
+
+    public static final RelOptRule WITH_LEFT_RIGHT_CALC_WINDOW_JOIN =
+            Config.EMPTY
+                    .withDescription("SimplifyWindowTableFunctionRuleWithLeftRightCalcWindowJoin")
+                    .as(Config.class)
+                    .withLeftRightCalcWindowJoin()
+                    .toRule();
+
+    public SimplifyWindowTableFunctionRule(Config config) {
+        super(config);
+    }
+
+    @Override
+    public boolean matches(RelOptRuleCall call) {
+        List<RelNode> rels = call.getRelList();
+        final RelNode node = rels.get(0);
+        if (node instanceof StreamPhysicalWindowRank) {
+            final int windowTVFIdx = rels.size() - 1;
+            final StreamPhysicalWindowTableFunction windowTVF =
+                    (StreamPhysicalWindowTableFunction) rels.get(windowTVFIdx);
+            return needSimplify(windowTVF);
+        } else if (node instanceof StreamPhysicalWindowJoin) {
+            RelNode leftWindowTVFRel;
+            if (call.rule == WITH_WINDOW_JOIN) {
+                leftWindowTVFRel = rels.get(2);
+            } else if (call.rule == WITH_RIGHT_CALC_WINDOW_JOIN) {
+                leftWindowTVFRel = rels.get(2);
+            } else if (call.rule == WITH_LEFT_CALC_WINDOW_JOIN) {
+                leftWindowTVFRel = rels.get(3);
+            } else if (call.rule == WITH_LEFT_RIGHT_CALC_WINDOW_JOIN) {
+                leftWindowTVFRel = rels.get(3);
+            } else {
+                throw new TableException("This should never happen. Please file an issue.");
+            }
+            final StreamPhysicalWindowTableFunction leftWindowTVF =
+                    (StreamPhysicalWindowTableFunction) leftWindowTVFRel;
+            final StreamPhysicalWindowTableFunction rightWindowTVF =
+                    (StreamPhysicalWindowTableFunction) rels.get(rels.size() - 1);
+            return needSimplify(leftWindowTVF) || needSimplify(rightWindowTVF);
+        } else {
+            throw new IllegalStateException(
+                    this.getClass().getName()
+                            + " matches a wrong relation tree: "
+                            + RelOptUtil.toString(node));
+        }
+    }
+
+    @Override
+    public void onMatch(RelOptRuleCall call) {
+        List<RelNode> rels = call.getRelList();
+        final RelNode node = rels.get(0);
+        if (node instanceof StreamPhysicalWindowRank) {
+            RelNode newTree = rebuild(rels);
+            call.transformTo(newTree);
+        } else if (node instanceof StreamPhysicalWindowJoin) {
+            RelNode newLeft;
+            RelNode newRight;
+            if (call.rule == WITH_WINDOW_JOIN) {

Review comment:
       This rule can be split to separate rules ? that will be more clear and we do not need
so many `if else` and constant numbers.

##########
File path: flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkStreamRuleSets.scala
##########
@@ -486,7 +486,13 @@ object FlinkStreamRuleSets {
     // incremental agg rule
     IncrementalAggregateRule.INSTANCE,
     // optimize window agg rule
-    TwoStageOptimizedWindowAggregateRule.INSTANCE
-  )
+    TwoStageOptimizedWindowAggregateRule.INSTANCE,
+    // simplify window tvf
+    SimplifyWindowTableFunctionRule.WITH_CALC_WINDOW_RANK,
+    SimplifyWindowTableFunctionRule.WITH_WINDOW_RANK,
+    SimplifyWindowTableFunctionRule.WITH_LEFT_RIGHT_CALC_WINDOW_JOIN,
+    SimplifyWindowTableFunctionRule.WITH_LEFT_CALC_WINDOW_JOIN,
+    SimplifyWindowTableFunctionRule.WITH_RIGHT_CALC_WINDOW_JOIN,
+    SimplifyWindowTableFunctionRule.WITH_WINDOW_JOIN)

Review comment:
       can those rules be put into `PHYSICAL_OPT_RULES` ?

##########
File path: flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowTableFunction.java
##########
@@ -39,15 +39,18 @@
 public class StreamExecWindowTableFunction extends ExecNodeBase<RowData>
         implements StreamExecNode<RowData> {
 
-    private final WindowingStrategy windowingStrategy;
+    private final TimeAttributeWindowingStrategy windowingStrategy;
+    private final Boolean emitPerRecord;

Review comment:
       nit: this class should support json ser/de. 




-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



Mime
View raw message