flink-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From GitBox <...@apache.org>
Subject [GitHub] [flink] hequn8128 commented on a change in pull request #9890: [FLINK-14272][python][table-planner-blink] Support Blink planner for Python UDF
Date Wed, 16 Oct 2019 12:03:09 GMT
hequn8128 commented on a change in pull request #9890: [FLINK-14272][python][table-planner-blink]
Support Blink planner for Python UDF
URL: https://github.com/apache/flink/pull/9890#discussion_r335423045
 
 

 ##########
 File path: flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecPythonCalc.scala
 ##########
 @@ -0,0 +1,106 @@
+/*
+ * 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.nodes.physical.stream
+
+import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
+import org.apache.calcite.rel.RelNode
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.Calc
+import org.apache.calcite.rex.{RexCall, RexInputRef, RexProgram}
+import org.apache.flink.api.dag.Transformation
+import org.apache.flink.streaming.api.transformations.OneInputTransformation
+import org.apache.flink.table.dataformat.BaseRow
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory
+import org.apache.flink.table.planner.delegation.StreamPlanner
+import org.apache.flink.table.planner.plan.nodes.common.CommonPythonCalc
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo
+
+import scala.collection.JavaConversions._
+
+/**
+  * Stream physical RelNode for Python ScalarFunctions.
+  */
+class StreamExecPythonCalc(
+    cluster: RelOptCluster,
+    traitSet: RelTraitSet,
+    inputRel: RelNode,
+    calcProgram: RexProgram,
+    outputRowType: RelDataType)
+  extends StreamExecCalcBase(
+    cluster,
+    traitSet,
+    inputRel,
+    calcProgram,
+    outputRowType)
+  with CommonPythonCalc {
+
+  override def copy(traitSet: RelTraitSet, child: RelNode, program: RexProgram): Calc = {
+    new StreamExecPythonCalc(cluster, traitSet, child, program, outputRowType)
+  }
+
+  private lazy val pythonRexCalls = calcProgram.getProjectList
+    .map(calcProgram.expandLocalRef)
+    .filter(_.isInstanceOf[RexCall])
+    .map(_.asInstanceOf[RexCall])
+    .toArray
+
+  private lazy val forwardedFields: Array[Int] = calcProgram.getProjectList
+    .map(calcProgram.expandLocalRef)
+    .filter(_.isInstanceOf[RexInputRef])
+    .map(_.asInstanceOf[RexInputRef].getIndex)
+    .toArray
+
+  private lazy val (pythonUdfInputOffsets, pythonFunctionInfos) =
+    extractPythonScalarFunctionInfos(pythonRexCalls)
+
+  override protected def translateToPlanInternal(
+      planner: StreamPlanner): Transformation[BaseRow] = {
+    val inputTransform = getInputNodes.get(0).translateToPlan(planner)
+      .asInstanceOf[Transformation[BaseRow]]
+
+    val inputLogicalTypes =
+      inputTransform.getOutputType.asInstanceOf[BaseRowTypeInfo].getLogicalTypes
+    val pythonOperatorInputTypeInfo = new BaseRowTypeInfo(inputLogicalTypes: _*)
+    val pythonOperatorResultTyeInfo = new BaseRowTypeInfo(
+      forwardedFields.map(inputLogicalTypes(_)) ++
+        pythonRexCalls.map(node => FlinkTypeFactory.toLogicalType(node.getType)): _*)
+
+    val pythonOperatorInputRowType = pythonOperatorInputTypeInfo.toRowType
+    val pythonOperatorOutputRowType = pythonOperatorResultTyeInfo.toRowType
+
+    val pythonOperator = getPythonScalarFunctionOperator(
+      pythonOperatorInputRowType, pythonOperatorOutputRowType,
+      pythonUdfInputOffsets, pythonFunctionInfos, forwardedFields)
+
+    val pythonInputTransform = new OneInputTransformation(
 
 Review comment:
   We need to re-range the field index after the python operator. In the result of python
operator, all results from python UDFs are appended at the end of the row and we need to recover
these fields.
   
   You can reproduce the error with a test case that put a field after a python udf, e.g.,
select("py_udf(a), a").
   
   
   

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

Mime
View raw message