[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

2022-12-20 Thread GitBox


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


##
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/EventTimeTemporalJoinRewriteRule.java:
##
@@ -0,0 +1,477 @@
+/*
+ * 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.table.api.TableException;
+import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalCalc;
+import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalJoin;
+import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalRel;
+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.flink.table.planner.plan.utils.TemporalTableJoinUtil;
+
+import 
org.apache.flink.shaded.curator5.org.apache.curator.shaded.com.google.common.collect.Lists;
+
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelRule;
+import org.apache.calcite.plan.hep.HepRelVertex;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.tools.RuleSet;
+import org.apache.calcite.tools.RuleSets;
+
+/**
+ * Traverses an event time temporal table join {@link RelNode} tree and update 
the right child to
+ * set {@link FlinkLogicalTableSourceScan}'s eventTimeSnapshot property to 
true which will prevent
+ * it generating a new StreamPhysicalChangelogNormalize later.
+ *
+ * the match patterns are as following(8 variants, the three `Calc` nodes 
are all optional):
+ *
+ * {@code
+ *Join (event time temporal)
+ *  /   \
+ * RelNode [Calc]
+ *   \
+ * Snapshot
+ *\
+ *  [Calc]
+ * \
+ * WatermarkAssigner
+ *  \
+ *[Calc]
+ *   \
+ *TableScan
+ * }
+ *
+ * Note: This rule can only be used in a separate {@link 
org.apache.calcite.plan.hep.HepProgram}
+ * after `LOGICAL_REWRITE` rule sets are applied for now.
+ */
+public class EventTimeTemporalJoinRewriteRule
+extends RelRule {
+
+public static final RuleSet EVENT_TIME_TEMPORAL_JOIN_REWRITE_RULES =
+RuleSets.ofList(
+Config.JOIN_CALC_SNAPSHOT_CALC_WMA_CALC_TS.toRule(),
+Config.JOIN_CALC_SNAPSHOT_CALC_WMA_TS.toRule(),
+Config.JOIN_CALC_SNAPSHOT_WMA_CALC_TS.toRule(),
+Config.JOIN_CALC_SNAPSHOT_WMA_TS.toRule(),
+Config.JOIN_SNAPSHOT_CALC_WMA_CALC_TS.toRule(),
+Config.JOIN_SNAPSHOT_CALC_WMA_TS.toRule(),
+Config.JOIN_SNAPSHOT_WMA_CALC_TS.toRule(),
+Config.JOIN_SNAPSHOT_WMA_TS.toRule());
+
+public EventTimeTemporalJoinRewriteRule(Config config) {
+super(config);
+}
+
+@Override
+public boolean matches(RelOptRuleCall call) {
+FlinkLogicalJoin join = call.rel(0);
+RexNode joinCondition = join.getCondition();
+// only matches event time temporal join
+return joinCondition != null
+&& 
TemporalTableJoinUtil.isEventTimeTemporalJoin(joinCondition);
+}
+
+@Override
+public void onMatch(RelOptRuleCall call) {
+FlinkLogicalJoin join = call.rel(0);
+FlinkLogicalRel joinRightChild = call.rel(2);
+RelNode newRight = transmitSnapshotRequirement(joinRightChild);
+call.transformTo(
+join.copy(join.getTraitSet(), 
Lists.newArrayList(join.getLeft(), newRight)));
+}
+
+private RelNode transmitSnapshotRequirement(RelNode node) {
+if (node instanceof FlinkLogicalCalc) {
+final FlinkLogicalCalc calc = (FlinkLogicalCalc) node;
+// filter is not allowed be

[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

2022-12-20 Thread GitBox


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


##
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/EventTimeTemporalJoinRewriteRule.java:
##
@@ -0,0 +1,484 @@
+/*
+ * 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.table.api.TableException;
+import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalCalc;
+import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalJoin;
+import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalRel;
+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.flink.table.planner.plan.utils.TemporalTableJoinUtil;
+
+import 
org.apache.flink.shaded.curator5.org.apache.curator.shaded.com.google.common.collect.Lists;
+
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelRule;
+import org.apache.calcite.plan.hep.HepRelVertex;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexNode;
+
+/**
+ * Traverses an event time temporal table join {@link RelNode} tree and update 
the right child to
+ * set {@link FlinkLogicalTableSourceScan}'s eventTimeSnapshot property to 
true which will prevent
+ * it generating a new StreamPhysicalChangelogNormalize later.
+ *
+ * above is the match patterns(8 variants, the three `Calc` nodes are all 
optional):
+ *
+ * {@code
+ *Join (event time temporal)
+ *  /   \
+ * RelNode [Calc]
+ *   \
+ * Snapshot
+ *\
+ *  [Calc]
+ * \
+ * WatermarkAssigner
+ *  \
+ *[Calc]
+ *   \
+ *TableScan
+ * }
+ *
+ * Note: This rule can only be used in a separate {@link 
org.apache.calcite.plan.hep.HepProgram}
+ * after `LOGICAL_REWRITE` rule sets are applied for now.
+ */
+public class EventTimeTemporalJoinRewriteRule
+extends RelRule {
+
+public static final RelOptRule JOIN_CALC_SNAPSHOT_CALC_WMA_CALC_TS =
+Config.JOIN_CALC_SNAPSHOT_CALC_WMA_CALC_TS.toRule();
+
+public static final RelOptRule JOIN_CALC_SNAPSHOT_CALC_WMA_TS =
+Config.JOIN_CALC_SNAPSHOT_CALC_WMA_TS.toRule();
+
+public static final RelOptRule JOIN_CALC_SNAPSHOT_WMA_CALC_TS =
+Config.JOIN_CALC_SNAPSHOT_WMA_CALC_TS.toRule();
+
+public static final RelOptRule JOIN_CALC_SNAPSHOT_WMA_TS =
+Config.JOIN_CALC_SNAPSHOT_WMA_TS.toRule();
+
+public static final RelOptRule JOIN_SNAPSHOT_CALC_WMA_CALC_TS =
+Config.JOIN_SNAPSHOT_CALC_WMA_CALC_TS.toRule();
+
+public static final RelOptRule JOIN_SNAPSHOT_CALC_WMA_TS =
+Config.JOIN_SNAPSHOT_CALC_WMA_TS.toRule();
+
+public static final RelOptRule JOIN_SNAPSHOT_WMA_CALC_TS =
+Config.JOIN_SNAPSHOT_WMA_CALC_TS.toRule();
+
+public static final RelOptRule JOIN_SNAPSHOT_WMA_TS = 
Config.JOIN_SNAPSHOT_WMA_TS.toRule();
+

Review Comment:
   yes, this will be more concise



##
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/EventTimeTemporalJoinRewriteRule.java:
##
@@ -0,0 +1,484 @@
+/*
+ * 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 o

[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

2022-12-18 Thread GitBox


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


##
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.
+ *
+ * Note: only snapshot on event time period will update the child {@link
+ * FlinkLogicalTableSourceScan}.
+ */
+public final class FlinkSnapshotConverter extends RelHomogeneousShuttle {

Review Comment:
   
![image](https://user-images.githubusercontent.com/3712895/208291077-978e6d8f-0c26-42e6-b6be-f0f2896c440e.png)
   
   a correction here: there're 8 variants not 4



-- 
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-unsubscr...@flink.apache.org

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



[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

2022-12-10 Thread GitBox


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


##
flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/TemporalJoinTest.scala:
##
@@ -570,6 +585,52 @@ class TemporalJoinTest extends TableTestBase {
 util.verifyExplainInsert(sql, ExplainDetail.CHANGELOG_MODE)
   }
 
+  @Test
+  def testTemporalJoinUpsertSourceWithPostFilter(): Unit = {
+val sqlQuery = "SELECT * " +
+  "FROM Orders AS o JOIN " +
+  "UpsertRates FOR SYSTEM_TIME AS OF o.rowtime AS r " +
+  "ON o.currency = r.currency WHERE valid = 'true'"
+
+util.verifyExplain(sqlQuery, ExplainDetail.CHANGELOG_MODE)
+  }
+
+  @Test
+  def testTemporalJoinUpsertSourceWithPreFilter(): Unit = {
+util.tableEnv.executeSql(s"""
+|CREATE TEMPORARY VIEW V1 AS
+|SELECT * FROM UpsertRates WHERE valid = 'true'
+|""".stripMargin)
+
+/**
+ * The problem is: there's exists a filter on an upsert changelog 
input(changelogMode=[I,UA,D]),
+ * the UB message must exists for correctness.
+ *
+ * Intermediate plan with modify kind:
+ * {{{
+ * +- TemporalJoin(joinType=[InnerJoin], ..., changelogMode=[I])
+ *:- Exchange(distribution=[hash[currency]], changelogMode=[I])
+ *  : +- WatermarkAssigner(rowtime=[rowtime], watermark=[rowtime], 
changelogMode=[I])
+ *: +- Calc(select=[amount, currency, rowtime, ... 
changelogMode=[I])
+ *  : +- TableSourceScan(table= Orders ... changelogMode=[I])
+ *+- Exchange(distribution=[hash[currency]], changelogMode=[I,UA,D])
+ *  +- Calc(select=[currency, ... where=[=(valid, _UTF-16LE'true')], 
changelogMode=[I,UA,D])
+ *+- WatermarkAssigner(rowtime=[rowtime], watermark=[rowtime], 
changelogMode=[I,UA,D])
+ *  +- TableSourceScan(table= UpsertRates, ... 
changelogMode=[I,UA,D])
+ * }}}
+ */
+
+val sqlQuery = "SELECT * " +
+  "FROM Orders AS o JOIN " +
+  "V1 FOR SYSTEM_TIME AS OF o.rowtime AS r " +
+  "ON o.currency = r.currency"
+
+expectExceptionThrown(
+  sqlQuery,
+  "Can't generate a valid execution plan for the given query",

Review Comment:
   Yes, this should be more clearer to users, I'll add validation and improve 
the error message.



##
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalTableSourceScan.scala:
##
@@ -46,20 +46,38 @@ class FlinkLogicalTableSourceScan(
 cluster: RelOptCluster,
 traitSet: RelTraitSet,
 hints: util.List[RelHint],
-relOptTable: TableSourceTable)
+val relOptTable: TableSourceTable,
+val eventTimeSnapshot: Boolean = false)

Review Comment:
   `eventTimeSnapshotRequired` maybe better here



-- 
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-unsubscr...@flink.apache.org

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



[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

2022-12-10 Thread GitBox


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


##
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.
+ *
+ * Note: only snapshot on event time period will update the child {@link
+ * FlinkLogicalTableSourceScan}.
+ */
+public final class FlinkSnapshotConverter extends RelHomogeneousShuttle {

Review Comment:
   @godfreyhe  I tried to implement multiple match patterns based on rule 
config, but compared to the shuttle approach, the code of rule implementation 
is several times more, because there are four possible pattern variants, as 
shown in the following figure(both calc nodes are optional):
   
![image](https://user-images.githubusercontent.com/3712895/206841489-9b4e363d-fd7b-4df5-8dad-c0a2487c276a.png)
   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-unsubscr...@flink.apache.org

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



[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

2022-11-09 Thread GitBox


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.
+ *
+ * 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 t