allisonwang-db commented on code in PR #42272: URL: https://github.com/apache/spark/pull/42272#discussion_r1289175446
########## python/docs/source/user_guide/sql/python_udtf.rst: ########## @@ -0,0 +1,222 @@ +.. 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. + +=========================================== +Python User-defined Table Functions (UDTFs) +=========================================== + +Spark 3.5 introduces Python user-defined table functions (UDTFs), a new type of user-defined function. +Unlike scalar functions that return a single result value, a UDTF is invoked in the FROM clause and returns +an entire relation as output. Each UDTF call can accept zero or more arguments. +These arguments can be scalar constant expressions or separate input relations. + +Implementing a Python UDTF +-------------------------- + +.. currentmodule:: pyspark.sql.functions + +To implement a Python UDTF, you can define a class implementing the methods: + +.. code-block:: python + + class PythonUDTF: + + def __init__(self) -> None: + """ + Initialize the user-defined table function (UDTF). + + This method serves as the default constructor and is called once when the + UDTF is instantiated on the executor side. + + Any class fields assigned in this method will be available for subsequent + calls to the `eval` and `terminate` methods. + + Notes + ----- + - This method does not accept any extra arguments. + - You cannot create or reference the Spark session within the UDTF. Any + attempt to do so will result in a serialization error. + """ + ... + + def eval(self, *args: Any) -> Iterator[Any]: + """ + Evaluate the function using the given input arguments. + + This method is required and must be implemented. + + The arguments provided to the UDTF call are mapped to the values in the + `*args` list sequentially. Each provided scalar expression maps to exactly + one value in this `*args` list. Each provided TABLE argument of N columns + maps to exactly N values in this `*args` list, in the order of the columns + as they appear in the table. + + This method is called on every input row, and can produce zero or more + output rows. Each element in the output tuple corresponds to one column + specified in the return type of the UDTF. + + Parameters + ---------- + *args : Any + Arbitrary positional arguments representing the input to the UDTF. + + Yields + ------ + tuple + A tuple representing a single row in the UDTF result relation. + Yield as many times as needed to produce multiple rows. + + Notes + ----- + - The result of the function must be a tuple representing a single row + in the UDTF result relation. + - UDTFs currently do not accept keyword arguments during the function call. + + Examples + -------- + >>> def eval(self, x: int, y: int) -> Iterator[Any]: + >>> yield x + y, x - y + >>> yield y + x, y - x + """ + ... + + def terminate(self) -> Iterator[Any]: + """ + Called when the UDTF has processed all input rows. + + This method is optional to implement and is useful for performing any + cleanup or finalization operations after the UDTF has finished processing + all rows in a partition. It can also be used to yield additional rows if needed. + + Yields + ------ + tuple + A tuple representing a single row in the UDTF result relation. + Yield this if you want to return additional rows during termination. + + Notes + ----- + - The UDTF's processing here is based on the partitioning of input table, + meaning that all input rows are rows within a specific partition of + the input relation. + + Examples + -------- + >>> def terminate(self) -> Iterator[Any]: + >>> yield "done", None + """ + ... + + +The return type of the UDTF defines the schema of the table it outputs. +It must be either a ``StructType`` or a DDL string representing a struct type. + +**Example UDTF Implementation:** + +Here is a simple example of a UDTF implementation: + +.. literalinclude:: ../../../../../examples/src/main/python/sql/udtf.py + :language: python + :lines: 38-40 + :dedent: 4 + + +**Instantiating the UDTF:** + +To make use of the UDTF, you'll first need to instantiate it using the `udtf` function: + +.. literalinclude:: ../../../../../examples/src/main/python/sql/udtf.py + :language: python + :lines: 42-51 + :dedent: 4 + +In this example, `SimpleUDTF` is instantiated with the return columns `c1` and `c2`, both of type `int`. + +**Using `udtf` as a Decorator:** + +An alternative way for implementing a UDTF is to use `udtf` as a decorator: + +.. literalinclude:: ../../../../../examples/src/main/python/sql/udtf.py + :language: python + :lines: 56-68 + :dedent: 4 + +For more detailed usage, please see :func:`udtf`. + + +Registering and Using Python UDTFs in SQL +----------------------------------------- + +Python UDTFs can also be registered and used in SQL queries. + +.. literalinclude:: ../../../../../examples/src/main/python/sql/udtf.py + :language: python + :lines: 72-98 + :dedent: 4 + + +Arrow Optimization +------------------ +Apache Arrow is an in-memory columnar data format used in Spark to efficiently transfer +data between JVM and Python processes. Apache Arrow is disabled by default for Python UDTFs. + +To enable Arrow optimization, set ``spark.sql.execution.pythonUDTF.arrow.enabled`` to ``true``. +Alternatively, you can specify the `useArrow` parameter when declaring the UDTF: + +.. literalinclude:: ../../../../../examples/src/main/python/sql/udtf.py + :language: python + :lines: 103-108 + :dedent: 4 + + +For more details, please see `Apache Arrow in PySpark <../arrow_pandas.rst>`_. + + +More Examples +------------- + +A Python UDTF with `__init__` and `terminate`: + +.. literalinclude:: ../../../../../examples/src/main/python/sql/udtf.py + :language: python + :lines: 113-132 + :dedent: 4 + + +A Python UDTF to generate a list of dates: + +.. literalinclude:: ../../../../../examples/src/main/python/sql/udtf.py + :language: python + :lines: 137-173 + :dedent: 4 + + +Advanced Features +----------------- + +TABLE input argument +~~~~~~~~~~~~~~~~~~~~ +Python UDTFs can also take a TABLE as input argument, and it can be used in conjunction +with scalar input arguments. +By default, you are allowed to have only one TABLE argument as input, primarily for +performance reasons. If you need to have more than one TABLE input argument, +you can enable this by setting ``spark.sql.tvf.allowMultipleTableArguments.enabled`` to ``true``. + +.. literalinclude:: ../../../../../examples/src/main/python/sql/udtf.py + :language: python + :lines: 178-197 + :dedent: 4 Review Comment: @ueshin @dtenedor I've added this TABLE argument support under the advanced feature section. We should add more examples here and in the udtf function docstring. -- 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. To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org