dtenedor commented on code in PR #42272: URL: https://github.com/apache/spark/pull/42272#discussion_r1282403569
########## python/docs/source/user_guide/sql/python_udtf.rst: ########## @@ -0,0 +1,140 @@ +.. 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 a new type of user-defined fucntion: Python user-defined table functions (UDTFs), +which take zero or more arguments and return a set of rows. Review Comment: ```suggestion wherein each invocation appears in the FROM clause and returns an entire relation as output instead of a single result value. Every UDTF call accepts zero or more arguments each comprising either a scalar constant expression or a separate input relation. ``` ########## python/docs/source/user_guide/sql/python_udtf.rst: ########## @@ -0,0 +1,140 @@ +.. 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 a new type of user-defined fucntion: Python user-defined table functions (UDTFs), +which take zero or more arguments and return a set of rows. + +Implementing a Python UDTF +-------------------------- + +.. currentmodule:: pyspark.sql.functions + +To implement a Python UDTF, you can implement this class: Review Comment: ```suggestion To implement a Python UDTF, you can define a class implementing these methods: ``` ########## python/docs/source/user_guide/sql/python_udtf.rst: ########## @@ -0,0 +1,140 @@ +.. 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 a new type of user-defined fucntion: Python user-defined table functions (UDTFs), +which take zero or more arguments and return a set of rows. + +Implementing a Python UDTF +-------------------------- + +.. currentmodule:: pyspark.sql.functions + +To implement a Python UDTF, you can implement this class: + +.. code-block:: python + + class PythonUDTF: + + def __init__(self) -> None: + """ + Initialize the user-defined table function (UDTF). + + This method is optional to implement and is called once when the UDTF is + instantiated. Use it to perform any initialization required for the UDTF. Review Comment: Can we also describe the UDTF class instance's lifetime here? For example, any class fields assigned here will be available for subsequent `eval` method call(s) to consume (either just one `eval` call for a UDTF call accepting only scalar constant arg(s) or several `eval` calls for a UDTF call accepting an input relation arg). ########## python/docs/source/user_guide/sql/python_udtf.rst: ########## @@ -0,0 +1,140 @@ +.. 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 a new type of user-defined fucntion: Python user-defined table functions (UDTFs), +which take zero or more arguments and return a set of rows. + +Implementing a Python UDTF +-------------------------- + +.. currentmodule:: pyspark.sql.functions + +To implement a Python UDTF, you can implement this class: + +.. code-block:: python + + class PythonUDTF: + + def __init__(self) -> None: + """ + Initialize the user-defined table function (UDTF). + + This method is optional to implement and is called once when the UDTF is + instantiated. Use it to perform any initialization required for the UDTF. + """ + ... + + def eval(self, *args: Any) -> Iterator[Any]: + """" + Evaluate the function using the given input arguments. + + This method is required to implement. + + Args: + *args: Arbitrary positional arguments representing the input + to the UDTF. + + Yields: + tuple: A tuple representing a single row in the UDTF result relation. + Yield thisas many times as needed to produce multiple rows. + + Note: + - The result must be a tuple. + - UDTFs do not accept keyword arguments on the calling side. + - Use "yield" to produce one row at a time for the UDTF result relation, + or use "return" to produce multiple rows for the UDTF result relation at once. + + Example: + def eval(self, x: int, y: int): + yield x + y, x - y Review Comment: can we also add an example with a combination of scalar constant arguments and a relation input argument, to show how the mapping from provided SQL arguments to the python `*args` works? Could we include a SQL query and its results with each example as well? ########## python/docs/source/user_guide/sql/python_udtf.rst: ########## @@ -0,0 +1,140 @@ +.. 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 a new type of user-defined fucntion: Python user-defined table functions (UDTFs), +which take zero or more arguments and return a set of rows. + +Implementing a Python UDTF +-------------------------- + +.. currentmodule:: pyspark.sql.functions + +To implement a Python UDTF, you can implement this class: + +.. code-block:: python + + class PythonUDTF: + + def __init__(self) -> None: + """ + Initialize the user-defined table function (UDTF). + + This method is optional to implement and is called once when the UDTF is + instantiated. Use it to perform any initialization required for the UDTF. + """ + ... + + def eval(self, *args: Any) -> Iterator[Any]: + """" + Evaluate the function using the given input arguments. Review Comment: Can we also describe the mapping between provided scalar argument(s) and the attributes of any provided relation argument with the ordering of these `*args` here? For example, we could mention that each provided scalar argument with index i in 0...N gets assigned to `*args` index i here, with the exception of at most one input relation argument at provided index j, and its attributes are injected one-by-one into the `*args` starting at index j? ########## python/docs/source/user_guide/sql/python_udtf.rst: ########## @@ -0,0 +1,140 @@ +.. 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 a new type of user-defined fucntion: Python user-defined table functions (UDTFs), +which take zero or more arguments and return a set of rows. + +Implementing a Python UDTF +-------------------------- + +.. currentmodule:: pyspark.sql.functions + +To implement a Python UDTF, you can implement this class: + +.. code-block:: python + + class PythonUDTF: + + def __init__(self) -> None: + """ + Initialize the user-defined table function (UDTF). + + This method is optional to implement and is called once when the UDTF is + instantiated. Use it to perform any initialization required for the UDTF. + """ + ... + + def eval(self, *args: Any) -> Iterator[Any]: + """" + Evaluate the function using the given input arguments. + + This method is required to implement. + + Args: + *args: Arbitrary positional arguments representing the input + to the UDTF. + + Yields: + tuple: A tuple representing a single row in the UDTF result relation. + Yield thisas many times as needed to produce multiple rows. + + Note: + - The result must be a tuple. + - UDTFs do not accept keyword arguments on the calling side. + - Use "yield" to produce one row at a time for the UDTF result relation, + or use "return" to produce multiple rows for the UDTF result relation at once. + + Example: + def eval(self, x: int, y: int): Review Comment: ```suggestion def eval(self, x: int, y: int) -> Iterator[Any]: ``` ########## python/docs/source/user_guide/sql/python_udtf.rst: ########## @@ -0,0 +1,140 @@ +.. 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 a new type of user-defined fucntion: Python user-defined table functions (UDTFs), +which take zero or more arguments and return a set of rows. + +Implementing a Python UDTF +-------------------------- + +.. currentmodule:: pyspark.sql.functions + +To implement a Python UDTF, you can implement this class: + +.. code-block:: python + + class PythonUDTF: + + def __init__(self) -> None: + """ + Initialize the user-defined table function (UDTF). + + This method is optional to implement and is called once when the UDTF is + instantiated. Use it to perform any initialization required for the UDTF. + """ + ... + + def eval(self, *args: Any) -> Iterator[Any]: + """" + Evaluate the function using the given input arguments. + + This method is required to implement. + + Args: + *args: Arbitrary positional arguments representing the input + to the UDTF. + + Yields: + tuple: A tuple representing a single row in the UDTF result relation. + Yield thisas many times as needed to produce multiple rows. + + Note: + - The result must be a tuple. + - UDTFs do not accept keyword arguments on the calling side. + - Use "yield" to produce one row at a time for the UDTF result relation, + or use "return" to produce multiple rows for the UDTF result relation at once. + + Example: + def eval(self, x: int, y: int): + yield x + y, x - y + """ + ... + + def terminate(self) -> Iterator[Any]: + """ + Called when the UDTF has processed all rows in a partition. Review Comment: We haven't really precisely defined what comprises a partition yet. Should we define it using the definitions from https://github.com/apache/spark/pull/42100 and https://github.com/apache/spark/pull/42174? Alternatively if these docs are targeting Spark 3.5 but those PRs are only going into `master`, we could simply define a partition here as either (1) just one `eval` call with the provided scalar argument(s), if any, or (2) several `eval` calls with an undefined subset of the rows from the input relation. Then we can expand it later. -- 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: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
