YuvalItzchakov commented on a change in pull request #15307:
URL: https://github.com/apache/flink/pull/15307#discussion_r606178294



##########
File path: 
flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/logical/PushFilterIntoTableSourceScanAcrossWatermarkRule.java
##########
@@ -0,0 +1,153 @@
+/*
+ * 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.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.TableScan;
+
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import 
org.apache.flink.table.connector.source.abilities.SupportsFilterPushDown;
+import org.apache.flink.table.planner.calcite.FlinkContext;
+import org.apache.flink.table.planner.calcite.FlinkRelFactories;
+import org.apache.flink.table.planner.plan.abilities.source.FilterPushDownSpec;
+import 
org.apache.flink.table.planner.plan.abilities.source.SourceAbilityContext;
+import org.apache.flink.table.planner.plan.abilities.source.SourceAbilitySpec;
+import 
org.apache.flink.table.planner.plan.nodes.calcite.LogicalWatermarkAssigner;
+import org.apache.flink.table.planner.plan.schema.TableSourceTable;
+import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil;
+import org.apache.flink.table.planner.plan.utils.RexNodeExtractor;
+import org.apache.flink.table.planner.plan.utils.RexNodeToExpressionConverter;
+import org.apache.flink.table.planner.utils.ShortcutUtils;
+
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.logical.LogicalFilter;
+import org.apache.calcite.rel.logical.LogicalTableScan;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.tools.RelBuilder;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.TimeZone;
+
+import scala.Tuple2;
+
+/**
+ * Pushes a {@link LogicalFilter} past a {@link LogicalWatermarkAssigner} and 
into {@link LogicalTableScan}
+ * in case the table scan implements {@link SupportsFilterPushDown}.
+ */
+public class PushFilterIntoTableSourceScanAcrossWatermarkRule extends 
PushFilterIntoSourceScanRuleBase {
+    public static final PushFilterIntoTableSourceScanAcrossWatermarkRule 
INSTANCE = new PushFilterIntoTableSourceScanAcrossWatermarkRule();
+
+    public PushFilterIntoTableSourceScanAcrossWatermarkRule() {
+        super(
+                operand(LogicalFilter.class, 
operand(LogicalWatermarkAssigner.class, operand(
+                        LogicalTableScan.class, none()))),
+                FlinkRelFactories.FLINK_REL_BUILDER(),
+                "PushFilterIntoTableSourceScanAcrossWatermarkRule");
+    }
+
+    @Override
+    public void onMatch(RelOptRuleCall call) {
+        LogicalFilter filter = call.rel(0);
+        LogicalWatermarkAssigner watermarkAssigner = call.rel(1);
+        LogicalTableScan scan = call.rel(2);
+
+        TableSourceTable oldTableSourceTable = 
scan.getTable().unwrap(TableSourceTable.class);
+
+        RelBuilder relBuilder = call.builder();
+        FlinkContext context = ShortcutUtils.unwrapContext(scan);
+        int maxCnfNodeCount = FlinkRelOptUtil.getMaxCnfNodeCount(scan);
+        RexNodeToExpressionConverter converter =
+                new RexNodeToExpressionConverter(
+                        relBuilder.getRexBuilder(),
+                        
filter.getInput().getRowType().getFieldNames().toArray(new String[0]),
+                        context.getFunctionCatalog(),
+                        context.getCatalogManager(),
+                        
TimeZone.getTimeZone(context.getTableConfig().getLocalTimeZone()));
+
+        Tuple2<RexNode[], RexNode[]> tuple2 =
+                RexNodeExtractor.extractConjunctiveConditions(
+                        filter.getCondition(),
+                        maxCnfNodeCount,
+                        relBuilder.getRexBuilder(),
+                        converter);
+        RexNode[] convertiblePredicates = tuple2._1;
+        RexNode[] unconvertedPredicates = tuple2._2;
+        if (convertiblePredicates.length == 0) {
+            // no condition can be translated to expression
+            return;
+        }
+
+        // record size before applyFilters for update statistics
+        int originPredicatesSize = convertiblePredicates.length;
+
+        // update DynamicTableSource
+        DynamicTableSource newTableSource = 
oldTableSourceTable.tableSource().copy();
+
+        SupportsFilterPushDown.Result result =
+                FilterPushDownSpec.apply(
+                        Arrays.asList(convertiblePredicates),
+                        newTableSource,
+                        SourceAbilityContext.from(scan));
+
+        relBuilder.push(scan);
+        List<RexNode> acceptedPredicates =
+                convertExpressionToRexNode(result.getAcceptedFilters(), 
relBuilder);
+        FilterPushDownSpec filterPushDownSpec = new 
FilterPushDownSpec(acceptedPredicates);
+
+        // record size after applyFilters for update statistics
+        int updatedPredicatesSize = result.getRemainingFilters().size();
+        // set the newStatistic newTableSource and extraDigests
+        TableSourceTable newTableSourceTable =
+                oldTableSourceTable.copy(
+                        newTableSource,
+                        getNewFlinkStatistic(
+                                oldTableSourceTable, originPredicatesSize, 
updatedPredicatesSize),
+                        getNewExtraDigests(result.getAcceptedFilters()),
+                        new SourceAbilitySpec[]{filterPushDownSpec});
+        LogicalTableScan newScan =
+                LogicalTableScan.create(scan.getCluster(), 
newTableSourceTable, scan.getHints());
+
+        LogicalWatermarkAssigner newWatermarkAssigner = 
LogicalWatermarkAssigner.create(
+                watermarkAssigner.getCluster(),
+                newScan,
+                watermarkAssigner.rowtimeFieldIndex(),
+                watermarkAssigner.watermarkExpr());
+
+        if (result.getRemainingFilters().isEmpty() && 
unconvertedPredicates.length == 0) {
+            call.transformTo(newWatermarkAssigner);
+        } else {
+            List<RexNode> remainingPredicates =
+                    convertExpressionToRexNode(result.getRemainingFilters(), 
relBuilder);
+            remainingPredicates.addAll(Arrays.asList(unconvertedPredicates));
+            RexNode remainingCondition = relBuilder.and(remainingPredicates);
+            Filter newFilter = filter.copy(
+                    filter.getTraitSet(),
+                    newWatermarkAssigner,
+                    remainingCondition);
+            call.transformTo(newFilter);

Review comment:
       It doesn't transpose them completely. There's a subset of predicates 
there are being pushed down and a part that's still left for the original 
filter to perform.




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

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


Reply via email to