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 07:51:15 UTC

[GitHub] [flink] lincoln-lil 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

lincoln-lil commented on code in PR #21219:
URL: https://github.com/apache/flink/pull/21219#discussion_r1018704994


##########
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:
   Good question! For my understanding though there's no strict rule for the implementations, but the shuttle utils are always the first choice(please correct me if I'm wrong, maybe @godfreyhe can give more guidance here). Back to this pr, the recursion is for the requirement transmition, and I didn't spent too much time to extend a shuttle to pass the requirement flag for visiting nodes(intuitively, the code will expand a lot), so I assume it's a reasonable choice for the specific issue here. Wish this can help explain your question.



##########
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:
   This is mainly for the long term goal of 'planner scala free'(so didn't change the private method TemporalJoinUtil#containsTemporalJoinCondition), but without adding a java util for this single method, we can add that util if more callers need this utility method or we want to refactor the 
    TemporalJoinUtil from scala to java, WDYT?



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