forked from apache/flink
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[FLINK-20702][python] Support map operation chained together in Pytho…
…n Table API This closes apache#14473.
- Loading branch information
1 parent
b474d28
commit febce35
Showing
9 changed files
with
318 additions
and
6 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
180 changes: 180 additions & 0 deletions
180
...k/src/main/java/org/apache/flink/table/planner/plan/rules/logical/PythonMapMergeRule.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,180 @@ | ||
/* | ||
* 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:https://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.planner.plan.nodes.logical.FlinkLogicalCalc; | ||
import org.apache.flink.table.planner.plan.utils.PythonUtil; | ||
|
||
import org.apache.calcite.plan.RelOptRule; | ||
import org.apache.calcite.plan.RelOptRuleCall; | ||
import org.apache.calcite.rel.core.Calc; | ||
import org.apache.calcite.rex.RexBuilder; | ||
import org.apache.calcite.rex.RexCall; | ||
import org.apache.calcite.rex.RexFieldAccess; | ||
import org.apache.calcite.rex.RexInputRef; | ||
import org.apache.calcite.rex.RexNode; | ||
import org.apache.calcite.rex.RexProgram; | ||
import org.apache.calcite.rex.RexProgramBuilder; | ||
|
||
import java.util.Collections; | ||
import java.util.List; | ||
import java.util.stream.Collectors; | ||
|
||
/** | ||
* Rule will merge Python {@link FlinkLogicalCalc} used in Map operation, Flatten {@link FlinkLogicalCalc} | ||
* and Python {@link FlinkLogicalCalc} used in Map operation together. | ||
*/ | ||
public class PythonMapMergeRule extends RelOptRule { | ||
|
||
public static final PythonMapMergeRule INSTANCE = new PythonMapMergeRule(); | ||
|
||
private PythonMapMergeRule() { | ||
super(operand(FlinkLogicalCalc.class, | ||
operand(FlinkLogicalCalc.class, | ||
operand(FlinkLogicalCalc.class, none()))), | ||
"PythonMapMergeRule"); | ||
} | ||
|
||
@Override | ||
public boolean matches(RelOptRuleCall call) { | ||
FlinkLogicalCalc topCalc = call.rel(0); | ||
FlinkLogicalCalc middleCalc = call.rel(1); | ||
FlinkLogicalCalc bottomCalc = call.rel(2); | ||
|
||
RexProgram topProgram = topCalc.getProgram(); | ||
List<RexNode> topProjects = topProgram.getProjectList() | ||
.stream() | ||
.map(topProgram::expandLocalRef) | ||
.collect(Collectors.toList()); | ||
|
||
if (topProjects.size() != 1 || PythonUtil.isNonPythonCall(topProjects.get(0)) || | ||
!PythonUtil.takesRowAsInput((RexCall) topProjects.get(0))) { | ||
return false; | ||
} | ||
|
||
RexProgram bottomProgram = bottomCalc.getProgram(); | ||
List<RexNode> bottomProjects = bottomProgram.getProjectList() | ||
.stream() | ||
.map(bottomProgram::expandLocalRef) | ||
.collect(Collectors.toList()); | ||
if (bottomProjects.size() != 1 || PythonUtil.isNonPythonCall(bottomProjects.get(0))) { | ||
return false; | ||
} | ||
|
||
RexProgram middleProgram = middleCalc.getProgram(); | ||
if (middleProgram.getCondition() != null) { | ||
return false; | ||
} | ||
|
||
List<RexNode> middleProjects = middleProgram.getProjectList() | ||
.stream() | ||
.map(middleProgram::expandLocalRef) | ||
.collect(Collectors.toList()); | ||
int inputRowFieldCount = middleProgram.getInputRowType() | ||
.getFieldList() | ||
.get(0) | ||
.getValue() | ||
.getFieldList().size(); | ||
|
||
return isFlattenCalc(middleProjects, inputRowFieldCount) && | ||
isTopCalcTakesWholeMiddleCalcAsInputs((RexCall) topProjects.get(0), middleProjects.size()); | ||
} | ||
|
||
private boolean isTopCalcTakesWholeMiddleCalcAsInputs(RexCall pythonCall, int inputColumnCount) { | ||
List<RexNode> pythonCallInputs = pythonCall.getOperands(); | ||
if (pythonCallInputs.size() != inputColumnCount) { | ||
return false; | ||
} | ||
for (int i = 0; i < pythonCallInputs.size(); i++) { | ||
RexNode input = pythonCallInputs.get(i); | ||
if (input instanceof RexInputRef) { | ||
if (((RexInputRef) input).getIndex() != i) { | ||
return false; | ||
} | ||
} else { | ||
return false; | ||
} | ||
} | ||
return true; | ||
} | ||
|
||
private boolean isFlattenCalc(List<RexNode> middleProjects, int inputRowFieldCount) { | ||
if (inputRowFieldCount != middleProjects.size()) { | ||
return false; | ||
} | ||
for (int i = 0; i < inputRowFieldCount; i++) { | ||
RexNode middleProject = middleProjects.get(i); | ||
if (middleProject instanceof RexFieldAccess) { | ||
RexFieldAccess rexField = ((RexFieldAccess) middleProject); | ||
if (rexField.getField().getIndex() != i) { | ||
return false; | ||
} | ||
RexNode expr = rexField.getReferenceExpr(); | ||
if (expr instanceof RexInputRef) { | ||
if (((RexInputRef) expr).getIndex() != 0) { | ||
return false; | ||
} | ||
} else { | ||
return false; | ||
} | ||
} else { | ||
return false; | ||
} | ||
} | ||
return true; | ||
} | ||
|
||
@Override | ||
public void onMatch(RelOptRuleCall call) { | ||
FlinkLogicalCalc topCalc = call.rel(0); | ||
FlinkLogicalCalc middleCalc = call.rel(1); | ||
FlinkLogicalCalc bottomCalc = call.rel(2); | ||
|
||
RexProgram topProgram = topCalc.getProgram(); | ||
List<RexCall> topProjects = topProgram.getProjectList() | ||
.stream() | ||
.map(topProgram::expandLocalRef) | ||
.map(x -> (RexCall) x) | ||
.collect(Collectors.toList()); | ||
RexCall topPythonCall = topProjects.get(0); | ||
|
||
// merge topCalc and middleCalc | ||
RexCall newPythonCall = topPythonCall.clone(topPythonCall.getType(), | ||
Collections.singletonList(RexInputRef.of(0, bottomCalc.getRowType()))); | ||
List<RexCall> topMiddleMergedProjects = Collections.singletonList(newPythonCall); | ||
FlinkLogicalCalc topMiddleMergedCalc = new FlinkLogicalCalc( | ||
middleCalc.getCluster(), | ||
middleCalc.getTraitSet(), | ||
bottomCalc, | ||
RexProgram.create( | ||
bottomCalc.getRowType(), | ||
topMiddleMergedProjects, | ||
null, | ||
Collections.singletonList("f0"), | ||
call.builder().getRexBuilder())); | ||
|
||
// merge bottomCalc | ||
RexBuilder rexBuilder = call.builder().getRexBuilder(); | ||
RexProgram mergedProgram = RexProgramBuilder.mergePrograms( | ||
topMiddleMergedCalc.getProgram(), bottomCalc.getProgram(), rexBuilder); | ||
Calc newCalc = topMiddleMergedCalc.copy( | ||
topMiddleMergedCalc.getTraitSet(), bottomCalc.getInput(), mergedProgram); | ||
call.transformTo(newCalc); | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
34 changes: 34 additions & 0 deletions
34
...st/resources/org/apache/flink/table/planner/plan/rules/logical/PythonMapMergeRuleTest.xml
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,34 @@ | ||
<?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:https://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> | ||
<TestCase name="testMapOperationsChained"> | ||
<Resource name="ast"> | ||
<![CDATA[ | ||
LogicalProject(_c0=[org$apache$flink$table$planner$runtime$utils$JavaUserDefinedScalarFunctions$RowPythonScalarFunction$92c809bc96452fbf4a7f26bbd91364c7(org$apache$flink$table$planner$runtime$utils$JavaUserDefinedScalarFunctions$RowPythonScalarFunction$92c809bc96452fbf4a7f26bbd91364c7(org$apache$flink$table$planner$runtime$utils$JavaUserDefinedScalarFunctions$RowPythonScalarFunction$92c809bc96452fbf4a7f26bbd91364c7($0, $1, $2).f0, org$apache$flink$table$planner$runtime$utils$JavaUserDefinedScalarFunctions$RowPythonScalarFunction$92c809bc96452fbf4a7f26bbd91364c7($0, $1, $2).f1).f0, org$apache$flink$table$planner$runtime$utils$JavaUserDefinedScalarFunctions$RowPythonScalarFunction$92c809bc96452fbf4a7f26bbd91364c7(org$apache$flink$table$planner$runtime$utils$JavaUserDefinedScalarFunctions$RowPythonScalarFunction$92c809bc96452fbf4a7f26bbd91364c7($0, $1, $2).f0, org$apache$flink$table$planner$runtime$utils$JavaUserDefinedScalarFunctions$RowPythonScalarFunction$92c809bc96452fbf4a7f26bbd91364c7($0, $1, $2).f1).f1).f0], _c1=[org$apache$flink$table$planner$runtime$utils$JavaUserDefinedScalarFunctions$RowPythonScalarFunction$92c809bc96452fbf4a7f26bbd91364c7(org$apache$flink$table$planner$runtime$utils$JavaUserDefinedScalarFunctions$RowPythonScalarFunction$92c809bc96452fbf4a7f26bbd91364c7(org$apache$flink$table$planner$runtime$utils$JavaUserDefinedScalarFunctions$RowPythonScalarFunction$92c809bc96452fbf4a7f26bbd91364c7($0, $1, $2).f0, org$apache$flink$table$planner$runtime$utils$JavaUserDefinedScalarFunctions$RowPythonScalarFunction$92c809bc96452fbf4a7f26bbd91364c7($0, $1, $2).f1).f0, org$apache$flink$table$planner$runtime$utils$JavaUserDefinedScalarFunctions$RowPythonScalarFunction$92c809bc96452fbf4a7f26bbd91364c7(org$apache$flink$table$planner$runtime$utils$JavaUserDefinedScalarFunctions$RowPythonScalarFunction$92c809bc96452fbf4a7f26bbd91364c7($0, $1, $2).f0, org$apache$flink$table$planner$runtime$utils$JavaUserDefinedScalarFunctions$RowPythonScalarFunction$92c809bc96452fbf4a7f26bbd91364c7($0, $1, $2).f1).f1).f1]) | ||
+- LogicalTableScan(table=[[default_catalog, default_database, source, source: [TestTableSource(a, b, c)]]]) | ||
]]> | ||
</Resource> | ||
<Resource name="optimized rel plan"> | ||
<![CDATA[ | ||
FlinkLogicalCalc(select=[f0.f0 AS _c0, f0.f1 AS _c1]) | ||
+- FlinkLogicalCalc(select=[pyFunc2(pyFunc2(pyFunc2(a, b, c))) AS f0]) | ||
+- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, source, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) | ||
]]> | ||
</Resource> | ||
</TestCase> | ||
</Root> |
67 changes: 67 additions & 0 deletions
67
...test/scala/org/apache/flink/table/planner/plan/rules/logical/PythonMapMergeRuleTest.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,67 @@ | ||
/* | ||
* 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:https://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.api.scala._ | ||
import org.apache.flink.table.api._ | ||
import org.apache.flink.table.planner.plan.nodes.FlinkConventions | ||
import org.apache.flink.table.planner.plan.optimize.program._ | ||
import org.apache.flink.table.planner.plan.rules.{FlinkBatchRuleSets, FlinkStreamRuleSets} | ||
import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedScalarFunctions.RowPythonScalarFunction | ||
import org.apache.flink.table.planner.utils.TableTestBase | ||
|
||
import org.apache.calcite.plan.hep.HepMatchOrder | ||
|
||
import org.junit.{Before, Test} | ||
|
||
/** | ||
* Test for [[PythonMapMergeRule]]. | ||
*/ | ||
class PythonMapMergeRuleTest extends TableTestBase { | ||
private val util = batchTestUtil() | ||
|
||
@Before | ||
def setup(): Unit = { | ||
val programs = new FlinkChainedProgram[BatchOptimizeContext]() | ||
programs.addLast( | ||
"logical", | ||
FlinkVolcanoProgramBuilder.newBuilder | ||
.add(FlinkBatchRuleSets.LOGICAL_OPT_RULES) | ||
.setRequiredOutputTraits(Array(FlinkConventions.LOGICAL)) | ||
.build()) | ||
programs.addLast( | ||
"logical_rewrite", | ||
FlinkHepRuleSetProgramBuilder.newBuilder | ||
.setHepRulesExecutionType(HEP_RULES_EXECUTION_TYPE.RULE_SEQUENCE) | ||
.setHepMatchOrder(HepMatchOrder.BOTTOM_UP) | ||
.add(FlinkStreamRuleSets.LOGICAL_REWRITE) | ||
.build()) | ||
util.replaceBatchProgram(programs) | ||
} | ||
|
||
@Test | ||
def testMapOperationsChained(): Unit = { | ||
val sourceTable = util.addTableSource[(Int, Int, Int)]("source", 'a, 'b, 'c) | ||
val func = new RowPythonScalarFunction("pyFunc2") | ||
val result = sourceTable.map(func(withColumns('*))) | ||
.map(func(withColumns('*))) | ||
.map(func(withColumns('*))) | ||
util.verifyRelPlan(result) | ||
} | ||
} |