jnh5y commented on code in PR #28162:
URL: https://github.com/apache/flink/pull/28162#discussion_r3307130919


##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/abilities/source/FilterPushDownSpec.java:
##########
@@ -110,11 +113,45 @@ public static SupportsFilterPushDown.Result apply(
         }
     }
 
+    /**
+     * Result of resolving {@link RexNode} predicates to {@link 
ResolvedExpression}s. Carries a
+     * reverse lookup keyed on {@link ResolvedExpression} <em>instance 
identity</em> so callers can
+     * walk the source's returned {@code accepted}/{@code remaining} lists and 
recover the original
+     * input {@link RexNode}s.
+     *
+     * <p>Sources are required to return back the same instances they receive 
(the apache pattern
+     * also used by {@link SupportsFilterPushDown}). A returned expression 
that is not in this map
+     * indicates the source rebuilt the expression and is treated as an error 
by the rule.
+     */
+    public static final class ResolvedPredicates {
+        /**
+         * Resolved expressions, in the same order as the input {@link 
RexNode}s and 1:1 with them.
+         * Pass these to the source's filter / metadata filter push-down 
method.
+         */
+        public final List<ResolvedExpression> resolvedExpressions;

Review Comment:
   Making this internal.



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/abilities/source/MetadataFilterPushDownSpec.java:
##########
@@ -79,10 +81,31 @@ public List<RexNode> getPredicates() {
     @Override
     public void apply(DynamicTableSource tableSource, SourceAbilityContext 
context) {
         // Use stored predicateRowType; context's row type may be narrowed by 
ProjectPushDownSpec.
-        MetadataFilterResult result =
-                applyMetadataFilters(predicates, predicateRowType, 
tableSource, context);
-        if (result.getAcceptedFilters().size() != predicates.size()) {
-            throw new TableException("All metadata predicates should be 
accepted here.");
+        MetadataFilterApplyResult applyResult =
+                applyMetadataFiltersWithLookup(predicates, predicateRowType, 
tableSource, context);
+        // Content-based round-trip: every input predicate must come back via 
the source's
+        // accepted list, identified by ResolvedExpression instance identity. 
Predicates that
+        // don't round-trip indicate the source either dropped them or rebuilt 
the expression
+        // (breaking the contract that the rule relied on at optimization 
time).
+        Map<ResolvedExpression, RexNode> reverseMap = 
applyResult.resolvedToInputRexNode;
+        IdentityHashMap<RexNode, Boolean> seen = new IdentityHashMap<>();

Review Comment:
   Switching to a set.



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to