You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2022/11/10 02:59:44 UTC

[GitHub] [flink] swuferhong commented on a diff in pull request #21219: [FLINK-29849][table-planner] Fix event time temporal join on an upsert source may produce incorrect execution plan

swuferhong commented on code in PR #21219:
URL: https://github.com/apache/flink/pull/21219#discussion_r1018581030


##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkSnapshotRequirementProgram.scala:
##########
@@ -0,0 +1,37 @@
+/*
+ * 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.optimize.program
+
+import org.apache.flink.table.planner.calcite.FlinkSnapshotConverter
+
+import org.apache.calcite.rel.RelNode
+
+/**
+ * A FlinkOptimizeProgram that deals with snapshot requirement from {@link FlinkLogicalSnapshot} to

Review Comment:
   Modify {@link FlinkLogicalSnapshot}  => [[FlinkLogicalSnapshot]]



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/FlinkFilterJoinRule.java:
##########
@@ -449,6 +455,33 @@ protected FlinkFilterIntoJoinRule(FlinkFilterIntoJoinRule.Config config) {
             super(config);
         }
 
+        @Override
+        public boolean matches(RelOptRuleCall call) {
+            Join join = call.rel(1);
+            return !isEventTimeTemporalJoin(join.getCondition()) && super.matches(call);
+        }

Review Comment:
   Can this `isEventTimeTemporalJoin` judgment use `TemporalJoinUtil.containsTemporalJoinCondition` or extracting `is event time temporal join judgment`  into this util class(` TemporalJoinUtil` ).



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/calcite/FlinkSnapshotConverter.java:
##########
@@ -0,0 +1,95 @@
+/*
+ * 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.calcite;
+
+import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalCalc;
+import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalSnapshot;
+import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalWatermarkAssigner;
+import org.apache.flink.table.planner.plan.schema.TimeIndicatorRelDataType;
+
+import org.apache.calcite.rel.RelHomogeneousShuttle;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.type.RelDataType;
+
+/**
+ * Traverses a {@link RelNode} tree and update the child node of {@link FlinkLogicalSnapshot} to set
+ * {@link FlinkLogicalTableSourceScan#eventTimeSnapshot} property.
+ *
+ * <p>Note: only snapshot on event time period will update the child {@link
+ * FlinkLogicalTableSourceScan}.
+ */
+public final class FlinkSnapshotConverter extends RelHomogeneousShuttle {
+
+    public static RelNode update(RelNode input) {
+        FlinkSnapshotConverter converter = new FlinkSnapshotConverter();
+        return input.accept(converter);
+    }
+
+    @Override
+    public RelNode visit(RelNode node) {
+        if (node instanceof FlinkLogicalSnapshot) {
+            final FlinkLogicalSnapshot snapshot = (FlinkLogicalSnapshot) node;
+            if (isEventTime(snapshot.getPeriod().getType())) {
+                final RelNode child = snapshot.getInput();
+                final RelNode newChild = transmitSnapshotRequirement(child);
+                if (newChild != child) {
+                    return snapshot.copy(snapshot.getTraitSet(), newChild, snapshot.getPeriod());
+                }
+            }
+            return snapshot;
+        }
+        return super.visit(node);
+    }
+
+    private boolean isEventTime(RelDataType period) {
+        if (period instanceof TimeIndicatorRelDataType) {
+            return ((TimeIndicatorRelDataType) period).isEventTime();
+        }
+        return false;
+    }
+
+    private RelNode transmitSnapshotRequirement(RelNode node) {
+        if (node instanceof FlinkLogicalCalc) {
+            final FlinkLogicalCalc calc = (FlinkLogicalCalc) node;
+            final RelNode child = calc.getInput();
+            final RelNode newChild = transmitSnapshotRequirement(child);
+            if (newChild != child) {
+                return calc.copy(calc.getTraitSet(), newChild, calc.getProgram());
+            }
+            return calc;
+        }
+        if (node instanceof FlinkLogicalWatermarkAssigner) {
+            final FlinkLogicalWatermarkAssigner wma = (FlinkLogicalWatermarkAssigner) node;
+            final RelNode child = wma.getInput();
+            final RelNode newChild = transmitSnapshotRequirement(child);

Review Comment:
   There is a puzzle generated in daily development that when to use `Shuttle` and when to write `recursion` when traversing the RelNode or RexNode node to obtain or modify a value? 



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