hequn8128 commented on a change in pull request #9907: 
[FLINK-14202][table][python] Optimize the execution plan for Python Calc when 
there is a condition
URL: https://github.com/apache/flink/pull/9907#discussion_r337315726
 
 

 ##########
 File path: 
flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PythonCalcSplitRule.scala
 ##########
 @@ -0,0 +1,321 @@
+/*
+ * 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.plan.RelOptRule.{any, operand}
+import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall}
+import org.apache.calcite.rex.{RexCall, RexInputRef, RexNode, RexProgram}
+import org.apache.calcite.sql.validate.SqlValidatorUtil
+import org.apache.flink.table.functions.FunctionLanguage
+import org.apache.flink.table.functions.ScalarFunction
+import org.apache.flink.table.planner.functions.utils.ScalarSqlFunction
+import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalCalc
+import org.apache.flink.table.planner.plan.utils.PythonUtil.containsFunctionOf
+import org.apache.flink.table.planner.plan.utils.{InputRefVisitor, 
RexDefaultVisitor}
+
+import scala.collection.JavaConverters._
+import scala.collection.JavaConversions._
+import scala.collection.mutable
+
+/**
+  * Rule that splits [[FlinkLogicalCalc]] into multiple [[FlinkLogicalCalc]]s.
+  * This is to ensure that the Python [[ScalarFunction]]s which could be
+  * executed in a batch are grouped into the same [[FlinkLogicalCalc]] node.
+  */
+abstract class PythonCalcSplitRuleBase(description: String)
+  extends RelOptRule(
+    operand(classOf[FlinkLogicalCalc], any),
+    description) {
+
+  override def onMatch(call: RelOptRuleCall): Unit = {
+    val calc: FlinkLogicalCalc = call.rel(0).asInstanceOf[FlinkLogicalCalc]
+    val input = calc.getInput
+    val rexBuilder = call.builder().getRexBuilder
+    val program = calc.getProgram
+    val extractedRexCalls = new mutable.ArrayBuffer[RexCall]()
+
+    val extractedFunctionOffset = input.getRowType.getFieldCount
+    val splitter = new ScalarFunctionSplitter(
+      extractedFunctionOffset,
+      extractedRexCalls,
+      convertPythonFunction(program))
+
+    val (bottomCalcCondition, topCalcCondition, topCalcProjects) = 
split(program, splitter)
+    val accessedFields = extractRefInputFields(
+      topCalcProjects, topCalcCondition, extractedFunctionOffset)
+
+    val bottomCalcProjects =
+      accessedFields.map(RexInputRef.of(_, input.getRowType)) ++ 
extractedRexCalls
+    val bottomCalcFieldNames = SqlValidatorUtil.uniquify(
+      accessedFields.map(i => input.getRowType.getFieldNames.get(i)).toSeq ++
+        extractedRexCalls.indices.map("f" + _),
+      rexBuilder.getTypeFactory.getTypeSystem.isSchemaCaseSensitive)
+
+    val bottomCalc = new FlinkLogicalCalc(
+      calc.getCluster,
+      calc.getTraitSet,
+      input,
+      RexProgram.create(
+        input.getRowType,
+        bottomCalcProjects.toList,
+        bottomCalcCondition.orNull,
+        bottomCalcFieldNames,
+        rexBuilder))
+
+    val inputRewriter = new 
ExtractedFunctionInputRewriter(extractedFunctionOffset, accessedFields)
+    val topCalc = new FlinkLogicalCalc(
+      calc.getCluster,
+      calc.getTraitSet,
+      bottomCalc,
+      RexProgram.create(
+        bottomCalc.getRowType,
+        topCalcProjects.map(_.accept(inputRewriter)),
+        topCalcCondition.map(_.accept(inputRewriter)).orNull,
+        calc.getRowType,
+        rexBuilder))
+
+    call.transformTo(topCalc)
+  }
+
+  /**
+    * Extracts the indices of the input fields referred by the specified 
projects and condition.
+    */
+  private def extractRefInputFields(
+      projects: Seq[RexNode],
+      condition: Option[RexNode],
+      inputFieldsCount: Int): Array[Int] = {
+    val visitor = new InputRefVisitor
+
+    // extract referenced input fields from projections
+    projects.foreach(exp => exp.accept(visitor))
+
+    // extract referenced input fields from condition
+    condition.foreach(_.accept(visitor))
+
+    // fields of indexes greater than inputFieldsCount is the extracted 
functions and
+    // should be filtered as they are not from the original input
+    visitor.getFields.filter(_ < inputFieldsCount)
+  }
+
+  /**
+   * Whether to convert Python functions.
+   */
+  def convertPythonFunction(program: RexProgram): Boolean
+
+  /**
+   * Splits the specified [[RexProgram]] using the specified 
[[ScalarFunctionSplitter]].
+   * It returns a triple of (bottom calc condition, top calc condition, top 
calc projects)
+   * as the split result.
+   */
+  def split(program: RexProgram, splitter: ScalarFunctionSplitter)
+      : (Option[RexNode], Option[RexNode], Seq[RexNode])
+}
+
+/**
+ * Rule that splits [[FlinkLogicalCalc]]s which contain Python functions in 
the condition
+ * into multiple [[FlinkLogicalCalc]]s. After this rule is applied, there will 
be no
+ * Python functions in the condition of the [[FlinkLogicalCalc]]s.
+ */
+object PythonCalcSplitFilterRule extends PythonCalcSplitRuleBase(
+  "PythonCalcSplitFilterRule") {
+
+  override def matches(call: RelOptRuleCall): Boolean = {
+    val calc: FlinkLogicalCalc = call.rel(0).asInstanceOf[FlinkLogicalCalc]
+    val condition = 
Option(calc.getProgram.getCondition).map(calc.getProgram.expandLocalRef)
+
+    // matches if it contains Python functions in condition
+    condition.nonEmpty && containsFunctionOf(condition.get, 
FunctionLanguage.PYTHON)
+  }
+
+  override def convertPythonFunction(program: RexProgram): Boolean = true
+
+  override def split(program: RexProgram, splitter: ScalarFunctionSplitter)
+      : (Option[RexNode], Option[RexNode], Seq[RexNode]) = {
+    (None, 
Option(program.getCondition).map(program.expandLocalRef(_).accept(splitter)),
+      program.getProjectList.map(program.expandLocalRef))
+  }
+}
+
+/**
+ * Rule that splits [[FlinkLogicalCalc]]s which contain both Java functions 
and Python functions
+ * in the projection into multiple [[FlinkLogicalCalc]]s. After this rule is 
applied, it will
+ * only contain Python functions or Java functions in the projection of each 
[[FlinkLogicalCalc]].
+ */
+object PythonCalcSplitProjectionRule extends PythonCalcSplitRuleBase(
+  "PythonCalcSplitProjectionRule") {
+
+  override def matches(call: RelOptRuleCall): Boolean = {
+    val calc: FlinkLogicalCalc = call.rel(0).asInstanceOf[FlinkLogicalCalc]
+    val condition = 
Option(calc.getProgram.getCondition).map(calc.getProgram.expandLocalRef)
+    val projects = 
calc.getProgram.getProjectList.map(calc.getProgram.expandLocalRef)
+
+    // matches if all the following conditions hold true:
+    // 1) it doesn't contain Python function in condition
+    // 2) it contains both Python functions and Java functions in the 
projection
+    !condition.exists(containsFunctionOf(_, FunctionLanguage.PYTHON)) &&
+    projects.exists(containsFunctionOf(_, FunctionLanguage.PYTHON)) &&
+      projects.exists(containsFunctionOf(_, FunctionLanguage.JVM))
+  }
+
+  override def convertPythonFunction(program: RexProgram): Boolean = {
+    program.getProjectList
+      .map(program.expandLocalRef)
+      .exists(containsFunctionOf(_, FunctionLanguage.JVM, recursive = false))
+  }
+
+  override def split(program: RexProgram, splitter: ScalarFunctionSplitter)
+      : (Option[RexNode], Option[RexNode], Seq[RexNode]) = {
+    (Option(program.getCondition).map(program.expandLocalRef), None,
+      program.getProjectList.map(program.expandLocalRef(_).accept(splitter)))
+  }
+}
+
+/**
+ * Rule that pushes the condition of [[FlinkLogicalCalc]]s before it for the
+ * [[FlinkLogicalCalc]]s which contain Python functions in the projection.
+ */
+object PythonCalcPushFilterRule extends PythonCalcSplitRuleBase(
+  "PythonCalcPushFilterRule") {
+
+  override def matches(call: RelOptRuleCall): Boolean = {
+    val calc: FlinkLogicalCalc = call.rel(0).asInstanceOf[FlinkLogicalCalc]
+    val projects = 
calc.getProgram.getProjectList.map(calc.getProgram.expandLocalRef)
+
+    // matches if all the following conditions hold true:
+    // 1) the condition is not null
+    // 2) it contains Python functions in the projection
+    calc.getProgram.getCondition != null &&
+      projects.exists(containsFunctionOf(_, FunctionLanguage.PYTHON))
+  }
+
+  override def convertPythonFunction(program: RexProgram): Boolean = false
+
+  override def split(program: RexProgram, splitter: ScalarFunctionSplitter)
+      : (Option[RexNode], Option[RexNode], Seq[RexNode]) = {
+    (Option(program.getCondition).map(program.expandLocalRef), None,
+      program.getProjectList.map(program.expandLocalRef))
+  }
+}
+
+/**
+ * Rule that ensures that it only contains [[RexInputRef]]s at the beginning of
+ * the project list and [[RexCall]]s at the end of the project list for 
[[FlinkLogicalCalc]]s
+ * which contain Python functions in the projection. This rule exists to keep
+ * DataStreamPythonCalc as simple as possible and ensures that it only needs to
+ * handle the Python function execution.
+ */
+object PythonCalcRewriteProjectRule extends PythonCalcSplitRuleBase(
 
 Review comment:
   Rename to `PythonCalcRewriteProjectionRule`? Make the name of 
`PythonCalcRewriteProjectRule` and `PythonCalcSplitProjectionRule` consistent.

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