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



##########
File path: 
flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/logical/PushFilterInCalcIntoTableSourceScanRule.java
##########
@@ -0,0 +1,144 @@
+/*
+ * 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.connector.source.abilities.SupportsFilterPushDown;
+import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalCalc;
+import 
org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalTableSourceScan;
+import org.apache.flink.table.planner.plan.schema.FlinkPreparingTableBase;
+import org.apache.flink.table.planner.plan.schema.TableSourceTable;
+
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.rel.core.Calc;
+import org.apache.calcite.rel.logical.LogicalCalc;
+import org.apache.calcite.rel.logical.LogicalTableScan;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexProgramBuilder;
+import org.apache.calcite.tools.RelBuilder;
+
+import java.util.List;
+import java.util.stream.Collectors;
+
+import scala.Tuple2;
+
+/** Pushes a filter condition from the {@link LogicalCalc} and into a {@link 
LogicalTableScan}. */

Review comment:
       /** Pushes a filter condition from the {@link FlinkLogicalCalc} and into 
a {@link FlinkLogicalTableSourceScan}. */

##########
File path: 
flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/logical/PushFilterInCalcIntoTableSourceScanRule.java
##########
@@ -0,0 +1,144 @@
+/*
+ * 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.connector.source.abilities.SupportsFilterPushDown;
+import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalCalc;
+import 
org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalTableSourceScan;
+import org.apache.flink.table.planner.plan.schema.FlinkPreparingTableBase;
+import org.apache.flink.table.planner.plan.schema.TableSourceTable;
+
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.rel.core.Calc;
+import org.apache.calcite.rel.logical.LogicalCalc;
+import org.apache.calcite.rel.logical.LogicalTableScan;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexProgramBuilder;
+import org.apache.calcite.tools.RelBuilder;
+
+import java.util.List;
+import java.util.stream.Collectors;
+
+import scala.Tuple2;
+
+/** Pushes a filter condition from the {@link LogicalCalc} and into a {@link 
LogicalTableScan}. */
+public class PushFilterInCalcIntoTableSourceScanRule extends 
PushFilterIntoSourceScanRuleBase {
+    public static final PushFilterInCalcIntoTableSourceScanRule INSTANCE =
+            new PushFilterInCalcIntoTableSourceScanRule();
+
+    public PushFilterInCalcIntoTableSourceScanRule() {
+        super(
+                operand(FlinkLogicalCalc.class, 
operand(FlinkLogicalTableSourceScan.class, none())),
+                "PushFilterInCalcIntoTableSourceScanRule");
+    }
+
+    @Override
+    public boolean matches(RelOptRuleCall call) {
+        if (!super.matches(call)) {
+            return false;
+        }
+
+        FlinkLogicalCalc calc = call.rel(0);
+        RexProgram originProgram = calc.getProgram();
+
+        if (originProgram.getCondition() == null) {
+            return false;
+        }
+
+        FlinkLogicalTableSourceScan scan = call.rel(1);
+        TableSourceTable tableSourceTable = 
scan.getTable().unwrap(TableSourceTable.class);
+        // we can not push filter twice
+        return canPushdownFilter(tableSourceTable);
+    }
+
+    @Override
+    public void onMatch(RelOptRuleCall call) {
+        FlinkLogicalCalc calc = call.rel(0);
+        FlinkLogicalTableSourceScan scan = call.rel(1);
+        TableSourceTable table = 
scan.getTable().unwrap(TableSourceTable.class);
+        pushFilterIntoScan(call, calc, scan, table);
+    }
+
+    private void pushFilterIntoScan(
+            RelOptRuleCall call,
+            Calc calc,
+            FlinkLogicalTableSourceScan scan,
+            FlinkPreparingTableBase relOptTable) {
+
+        RexProgram originProgram = calc.getProgram();
+
+        RelBuilder relBuilder = call.builder();
+        Tuple2<RexNode[], RexNode[]> extractedPredicates =
+                extractPredicates(
+                        
originProgram.getInputRowType().getFieldNames().toArray(new String[0]),
+                        
originProgram.expandLocalRef(originProgram.getCondition()),
+                        scan,
+                        relBuilder.getRexBuilder());
+
+        RexNode[] convertiblePredicates = extractedPredicates._1;
+        if (convertiblePredicates.length == 0) {
+            // no condition can be translated to expression
+            return;
+        }
+
+        Tuple2<SupportsFilterPushDown.Result, TableSourceTable> 
pushdownResultWithScan =
+                resolveFiltersAndCreateTableSourceTable(
+                        convertiblePredicates,
+                        relOptTable.unwrap(TableSourceTable.class),
+                        scan,
+                        relBuilder);
+
+        SupportsFilterPushDown.Result result = pushdownResultWithScan._1;
+        TableSourceTable tableSourceTable = pushdownResultWithScan._2;
+
+        FlinkLogicalTableSourceScan newScan =
+                FlinkLogicalTableSourceScan.create(scan.getCluster(), 
tableSourceTable);
+
+        // build new calc program
+        RexProgramBuilder programBuilder =

Review comment:
       nit: we can create a RexProgramBuilder instance from a given RexProgram 
via `RexProgramBuilder programBuilder =RexProgramBuilder.forProgram(...)` and 
just call `RexProgramBuilder#clearCondition` or RexProgramBuilder#addCondition` 
based on the condition.

##########
File path: 
flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest.xml
##########
@@ -0,0 +1,35 @@
+<?xml version="1.0" ?>
+<!--
+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.
+-->
+<Root>

Review comment:
       This file can be removed




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