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/14 15:21:32 UTC

[GitHub] [flink] lincoln-lil commented on a diff in pull request #20745: [FLINK-28988] Don't push above filters down into the right table for temporal join

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


##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/FlinkFilterJoinRule.java:
##########
@@ -128,7 +129,9 @@ protected void perform(RelOptRuleCall call, Filter filter, Join join) {
                 joinType,
                 true,
                 !joinType.generatesNullsOnLeft(),
-                !joinType.generatesNullsOnRight(),
+                !joinType.generatesNullsOnRight()
+                        && !TemporalJoinUtil.containsInitialTemporalJoinCondition(

Review Comment:
   the filter added here and line 176 below are not enough to prevent pushing down a filter into the right   side input of event time temporal join, because this rule will also perform pushing both above and join filters to opposite side by transitive rule, e.g., think of such an query:
   
   ```sql
   SELECT *      
   FROM Orders AS o JOIN
   rates_last_row_rowtime FOR SYSTEM_TIME AS OF o.rowtime AS r
   ON o.currency = r.currency AND o.currency = 10
   ```
   though the local predicate 'o.currency = 10' is releated to left side, but by this transitive optimization, it will push a new filter 'r.currency = 10' to the right side, this will break our target, so we need to figure out a way to cover this case



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