You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2020/12/09 06:32:29 UTC

[GitHub] [flink] dianfu commented on a change in pull request #14317: [FLINK-20482][python] Support General Python UDF for Map Operation in Python Table API

dianfu commented on a change in pull request #14317:
URL: https://github.com/apache/flink/pull/14317#discussion_r538946963



##########
File path: flink-python/pyflink/table/table.py
##########
@@ -759,6 +759,27 @@ def drop_columns(self, *fields: Union[str, Expression]) -> 'Table':
             assert isinstance(fields[0], str)
             return Table(self._j_table.dropColumns(fields[0]), self._t_env)
 
+    def map(self, func: Union[str, Expression]) -> 'Table':
+        """
+        Performs a map operation with an user-defined scalar function.
+
+        Example:
+        ::
+
+            >>> add = udf(lambda x: Row(x + 1, x *x), result_type=DataTypes.Row(
+            ... [DataTypes.FIELD("a", DataTypes.INT()), DataTypes.FIELD("b", DataTypes.INT())]))
+            >>> table_env.create_temporary_function("add", add)
+            >>> tab.map(add(tab.a)).alias("a, b")
+            >>> tab.map("add(a)").alias("a, b")
+
+        :param func: user-defined scalar function.
+        :return: The result table.

Review comment:
       .. versionadded:: 1.13.0

##########
File path: flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PythonCalcSplitRule.scala
##########
@@ -333,6 +387,18 @@ private class ScalarFunctionSplitter(
     extractedRexNodes.append(node)
     newNode
   }
+
+  private def getExtractRexFieldNode(node: RexFieldAccess, rexCallIndex: Int): RexNode = {

Review comment:
       ```suggestion
     private def getExtractedRexFieldAccess(node: RexFieldAccess, rexCallIndex: Int): RexNode = {
   ```

##########
File path: flink-python/pyflink/table/tests/test_row_based_operation.py
##########
@@ -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://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.
+################################################################################
+
+from pyflink.common import Row
+from pyflink.table.types import DataTypes
+from pyflink.table.udf import udf
+from pyflink.testing import source_sink_utils
+from pyflink.testing.test_case_utils import PyFlinkBlinkBatchTableTestCase, \
+    PyFlinkBlinkStreamTableTestCase
+
+
+class RowBasedOperationTests(object):
+    def test_map(self):
+        t = self.t_env.from_elements(
+            [(1, 2, 3), (2, 1, 3), (1, 5, 4), (1, 8, 6), (2, 3, 4)],
+            DataTypes.ROW(
+                [DataTypes.FIELD("a", DataTypes.TINYINT()),
+                 DataTypes.FIELD("b", DataTypes.SMALLINT()),
+                 DataTypes.FIELD("c", DataTypes.INT())]))
+
+        table_sink = source_sink_utils.TestAppendSink(
+            ['a', 'b'],
+            [DataTypes.BIGINT(), DataTypes.BIGINT()])
+        self.t_env.register_table_sink("Results", table_sink)
+
+        func = udf(lambda x: Row(x + 1, x * x), result_type=DataTypes.ROW(
+            [DataTypes.FIELD("a", DataTypes.BIGINT()),
+             DataTypes.FIELD("b", DataTypes.BIGINT())]))
+
+        t.map(func(t.b)).alias("a", "b").execute_insert("Results").wait()

Review comment:
       What about adding the following test cases?
   - test case for two continuous map
   - test case for pandas udf

##########
File path: flink-python/pyflink/table/table.py
##########
@@ -759,6 +759,27 @@ def drop_columns(self, *fields: Union[str, Expression]) -> 'Table':
             assert isinstance(fields[0], str)
             return Table(self._j_table.dropColumns(fields[0]), self._t_env)
 
+    def map(self, func: Union[str, Expression]) -> 'Table':
+        """
+        Performs a map operation with an user-defined scalar function.
+
+        Example:
+        ::
+
+            >>> add = udf(lambda x: Row(x + 1, x *x), result_type=DataTypes.Row(
+            ... [DataTypes.FIELD("a", DataTypes.INT()), DataTypes.FIELD("b", DataTypes.INT())]))
+            >>> table_env.create_temporary_function("add", add)
+            >>> tab.map(add(tab.a)).alias("a, b")
+            >>> tab.map("add(a)").alias("a, b")

Review comment:
       we prefer the expression style and so I guess we could remove the str style example

##########
File path: flink-python/pyflink/table/table.py
##########
@@ -759,6 +759,27 @@ def drop_columns(self, *fields: Union[str, Expression]) -> 'Table':
             assert isinstance(fields[0], str)
             return Table(self._j_table.dropColumns(fields[0]), self._t_env)
 
+    def map(self, func: Union[str, Expression]) -> 'Table':
+        """
+        Performs a map operation with an user-defined scalar function.

Review comment:
       ```suggestion
           Performs a map operation with a user-defined scalar function.
   ```

##########
File path: flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PythonCalcSplitRule.scala
##########
@@ -333,6 +387,18 @@ private class ScalarFunctionSplitter(
     extractedRexNodes.append(node)
     newNode
   }
+
+  private def getExtractRexFieldNode(node: RexFieldAccess, rexCallIndex: Int): RexNode = {
+    val pythonCall: RexCall = program.expandLocalRef(
+      node.getReferenceExpr.asInstanceOf[RexLocalRef]).asInstanceOf[RexCall]
+    if (!fieldsRexCall.contains(rexCallIndex)) {
+      extractedRexNodes.append(pythonCall)
+      fieldsRexCall += rexCallIndex -> (extractedFunctionOffset + extractedRexNodes.length -1)

Review comment:
       ```suggestion
         fieldsRexCall += rexCallIndex -> (extractedFunctionOffset + extractedRexNodes.length - 1)
   ```




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