hequn8128 commented on a change in pull request #11242: 
[FLINK-16007][table-planner][table-planner-blink][python] Add 
PythonCorrelateSplitRule to push down the Java Calls contained in Python 
Correlate node
URL: https://github.com/apache/flink/pull/11242#discussion_r385994214
 
 

 ##########
 File path: 
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/plan/rules/logical/PythonCorrelateSplitRule.java
 ##########
 @@ -0,0 +1,216 @@
+/*
+ * 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.plan.rules.logical;
+
+import org.apache.flink.table.plan.nodes.logical.FlinkLogicalCalc;
+import org.apache.flink.table.plan.nodes.logical.FlinkLogicalCorrelate;
+import org.apache.flink.table.plan.nodes.logical.FlinkLogicalTableFunctionScan;
+import org.apache.flink.table.plan.util.CorrelateUtil;
+import org.apache.flink.table.plan.util.PythonUtil;
+
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.hep.HepRelVertex;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLocalRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexProgramBuilder;
+import org.apache.calcite.sql.validate.SqlValidatorUtil;
+
+import java.util.LinkedList;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import scala.Option;
+import scala.collection.Iterator;
+import scala.collection.mutable.ArrayBuffer;
+
+/**
+ * Rule will split the {@link FlinkLogicalTableFunctionScan} which includes 
java calls into a
+ * {@link FlinkLogicalCalc} which will be the left input of the new  {@link 
FlinkLogicalCorrelate}
+ * and a new {@link FlinkLogicalTableFunctionScan} without java calls.
+ */
+public class PythonCorrelateSplitRule extends RelOptRule {
+
+       public static final PythonCorrelateSplitRule INSTANCE = new 
PythonCorrelateSplitRule();
+
+       private PythonCorrelateSplitRule() {
+               super(operand(FlinkLogicalCorrelate.class, any()), 
"PythonCorrelateSplitRule");
+       }
+
+       private FlinkLogicalTableFunctionScan createNewScan(
+               FlinkLogicalTableFunctionScan scan,
+               ScalarFunctionSplitter splitter) {
+               RexCall rightRexCall = (RexCall) scan.getCall();
+               // extract java funcs.
+               List<RexNode> rightCalcProjects = rightRexCall
+                       .getOperands()
+                       .stream()
+                       .map(x -> x.accept(splitter))
+                       .collect(Collectors.toList());
+
+               RexCall newRightRexCall = 
rightRexCall.clone(rightRexCall.getType(), rightCalcProjects);
+               return new FlinkLogicalTableFunctionScan(
+                       scan.getCluster(),
+                       scan.getTraitSet(),
+                       scan.getInputs(),
+                       newRightRexCall,
+                       scan.getElementType(),
+                       scan.getRowType(),
+                       scan.getColumnMappings());
+       }
+
+       @Override
+       public boolean matches(RelOptRuleCall call) {
+               FlinkLogicalCorrelate correlate = call.rel(0);
+               RelNode right = ((HepRelVertex) 
correlate.getRight()).getCurrentRel();
+               FlinkLogicalTableFunctionScan pythonTableFuncScan;
+               if (right instanceof FlinkLogicalTableFunctionScan) {
+                       pythonTableFuncScan = (FlinkLogicalTableFunctionScan) 
right;
+               } else if (right instanceof FlinkLogicalCalc) {
+                       Option<FlinkLogicalTableFunctionScan> scan = 
CorrelateUtil
+                               .getTableFunctionScan((FlinkLogicalCalc) right);
+                       if (scan.isEmpty()) {
+                               return false;
+                       }
+                       pythonTableFuncScan = scan.get();
+               } else {
+                       return false;
+               }
+               RexNode rexNode = pythonTableFuncScan.getCall();
+               if (rexNode instanceof RexCall) {
+                       RexCall pythonTableFuncRexCall = (RexCall) rexNode;
+                       return PythonUtil.isPythonCall(pythonTableFuncRexCall) 
&&
+                               
pythonTableFuncRexCall.getOperands().stream().anyMatch(PythonUtil::isNonPythonCall);
+               }
 
 Review comment:
   Change to
   ```
                RexNode rexNode = pythonTableFuncScan.getCall();
                if (rexNode instanceof RexCall) {
                        return PythonUtil.isPythonCall(rexNode) && 
PythonUtil.containsNonPythonCall(rexNode);
                }
   ```

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


With regards,
Apache Git Services

Reply via email to