You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "allisonwang-db (via GitHub)" <gi...@apache.org> on 2023/08/17 21:03:02 UTC

[GitHub] [spark] allisonwang-db commented on a diff in pull request #42420: [SPARK-44748][SQL] Query execution for the PARTITION BY clause in UDTF TABLE arguments

allisonwang-db commented on code in PR #42420:
URL: https://github.com/apache/spark/pull/42420#discussion_r1297730442


##########
python/pyspark/worker.py:
##########
@@ -573,9 +574,73 @@ def read_udtf(pickleSer, infile, eval_type):
             f"The return type of a UDTF must be a struct type, but got {type(return_type)}."
         )
 
+    class UDTFWithPartitions:
+        """
+        This implements the logic of a UDTF that accepts an input TABLE argument with one or more
+        PARTITION BY expressions.

Review Comment:
   It would be extremely helpful if we can add an example here. Maybe something like 
   -- t: [c1, c2] 
   SELECT * FROM my_udtf(TABLE (t) PARTITION BY .. ORDER BY ..)
   partition_child_indexes in this case will be ...



##########
python/pyspark/worker.py:
##########
@@ -573,9 +574,73 @@ def read_udtf(pickleSer, infile, eval_type):
             f"The return type of a UDTF must be a struct type, but got {type(return_type)}."
         )
 
+    class UDTFWithPartitions:
+        """
+        This implements the logic of a UDTF that accepts an input TABLE argument with one or more
+        PARTITION BY expressions.
+
+        Parameters
+        ----------
+        create_udtf: function
+            Function to create a new instance of the UDTF to be invoked.
+        partition_child_indexes: list
+            List of integers identifying zero-based indexes of the columns of the input table that
+            contain projected partitioning expressions. This class will inspect these values for
+            each pair of consecutive input rows. When they change, this indicates the boundary
+            between two partitions, and we will invoke the 'terminate' method on the UDTF class
+            instance and then destroy it and create a new one to implement the desired partitioning
+            semantics.
+        """
+        def __init__(self, create_udtf, partition_child_indexes):

Review Comment:
   nit: can we add type hint here?



##########
python/pyspark/worker.py:
##########
@@ -573,9 +574,73 @@ def read_udtf(pickleSer, infile, eval_type):
             f"The return type of a UDTF must be a struct type, but got {type(return_type)}."
         )
 
+    class UDTFWithPartitions:
+        """
+        This implements the logic of a UDTF that accepts an input TABLE argument with one or more
+        PARTITION BY expressions.
+
+        Parameters
+        ----------
+        create_udtf: function
+            Function to create a new instance of the UDTF to be invoked.
+        partition_child_indexes: list
+            List of integers identifying zero-based indexes of the columns of the input table that
+            contain projected partitioning expressions. This class will inspect these values for
+            each pair of consecutive input rows. When they change, this indicates the boundary
+            between two partitions, and we will invoke the 'terminate' method on the UDTF class
+            instance and then destroy it and create a new one to implement the desired partitioning
+            semantics.
+        """
+        def __init__(self, create_udtf, partition_child_indexes):
+            self._create_udtf = create_udtf
+            self._udtf = create_udtf()
+            self._prev_arguments = None
+            self._partition_child_indexes = partition_child_indexes
+
+        def eval(self, *args, **kwargs):
+            changed_partitions = self._check_partition_boundaries(
+                list(args) + list(kwargs.values()))
+            if changed_partitions:
+                if self._udtf.terminate is not None:
+                    result = self._udtf.terminate()
+                    if result is not None:
+                        for row in result:
+                            yield row
+                self._udtf = self._create_udtf()
+            if self._udtf.eval is not None:
+                result = self._udtf.eval(*args, **kwargs)
+                if result is not None:
+                    for row in result:
+                        yield row
+
+        def terminate(self):
+            if self._udtf.terminate is not None:
+                return self._udtf.terminate()
+
+        def _check_partition_boundaries(self, arguments):
+            result = False

Review Comment:
   nit: can we add type hinting for the function input and output, and docstring comments? And also for _get_table_arg?



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala:
##########
@@ -2118,11 +2121,32 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor
                   tableArgs.size)
               }
               val alias = SubqueryAlias.generateSubqueryName(s"_${tableArgs.size}")
+              // Propagate the column indexes for TABLE arguments to the PythonUDTF instance.
+              val tvfWithTableColumnIndexes: LogicalPlan = tvf match {
+                case g @ Generate(p: PythonUDTF, _, _, _, _, _) =>
+                  functionTableSubqueryArgs.headOption.map { tableArg =>
+                    val indexes = PythonUDTFPartitionColumnIndexes(
+                      tableArg.partitioningExpressionIndexes)
+                    g.copy(generator = p.copy(pythonUDTFPartitionColumnIndexes = Some(indexes)))
+                  }.getOrElse {
+                    g
+                  }
+                case g @ Generate(p: UnresolvedPolymorphicPythonUDTF, _, _, _, _, _) =>
+                  functionTableSubqueryArgs.headOption.map { tableArg =>
+                    val indexes = PythonUDTFPartitionColumnIndexes(
+                      tableArg.partitioningExpressionIndexes)
+                    g.copy(generator = p.copy(pythonUDTFPartitionColumnIndexes = Some(indexes)))
+                  }.getOrElse {
+                    g

Review Comment:
   We can combine this with the one above:
   ```
   p : (PythonUDTF | UnresolvedPolymorphicPythonUDTF)
   ```



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchEvalPythonUDTFExec.scala:
##########
@@ -134,6 +134,13 @@ object PythonUDTFRunner {
             dataOut.writeBoolean(false)
         }
     }
+    udtf.pythonUDTFPartitionColumnIndexes match {
+      case Some(partitionColumnIndexes) =>
+        dataOut.writeInt(partitionColumnIndexes.partitionChildIndexes.length)
+        partitionColumnIndexes.partitionChildIndexes.foreach(dataOut.writeInt)

Review Comment:
   can we assert partitionColumnIndexes.partitionChildIndexes.length > 0 here?



##########
python/pyspark/worker.py:
##########
@@ -573,9 +574,73 @@ def read_udtf(pickleSer, infile, eval_type):
             f"The return type of a UDTF must be a struct type, but got {type(return_type)}."
         )
 
+    class UDTFWithPartitions:
+        """
+        This implements the logic of a UDTF that accepts an input TABLE argument with one or more
+        PARTITION BY expressions.
+
+        Parameters
+        ----------
+        create_udtf: function
+            Function to create a new instance of the UDTF to be invoked.
+        partition_child_indexes: list
+            List of integers identifying zero-based indexes of the columns of the input table that
+            contain projected partitioning expressions. This class will inspect these values for
+            each pair of consecutive input rows. When they change, this indicates the boundary
+            between two partitions, and we will invoke the 'terminate' method on the UDTF class
+            instance and then destroy it and create a new one to implement the desired partitioning
+            semantics.
+        """
+        def __init__(self, create_udtf, partition_child_indexes):
+            self._create_udtf = create_udtf
+            self._udtf = create_udtf()
+            self._prev_arguments = None
+            self._partition_child_indexes = partition_child_indexes
+
+        def eval(self, *args, **kwargs):
+            changed_partitions = self._check_partition_boundaries(
+                list(args) + list(kwargs.values()))
+            if changed_partitions:
+                if self._udtf.terminate is not None:
+                    result = self._udtf.terminate()
+                    if result is not None:
+                        for row in result:
+                            yield row

Review Comment:
   I wonder if this will cause issues when we join the output rows from the UDTF with the input rows. Here is how we join the rows:
   ```
   input_row_1 -> [eval_output_row_1, eval_output_row_2, .. ]
   input_row_2 - >[eval_output_row_1, ..]
   (now assume we consumed all input rows in a partition, input_row_2 is the last one)
   (we use the last row in a partition to join with the rows generated by terminate)
   input_row_2 -> [terminate_output_row_1, terminate_output_row_2, ...]
   ```
   If we directly yield rows from `terminate` here, will they be joined with the input row from the next partition?



##########
python/pyspark/worker.py:
##########
@@ -573,9 +574,73 @@ def read_udtf(pickleSer, infile, eval_type):
             f"The return type of a UDTF must be a struct type, but got {type(return_type)}."
         )
 
+    class UDTFWithPartitions:
+        """
+        This implements the logic of a UDTF that accepts an input TABLE argument with one or more
+        PARTITION BY expressions.
+
+        Parameters
+        ----------
+        create_udtf: function
+            Function to create a new instance of the UDTF to be invoked.
+        partition_child_indexes: list
+            List of integers identifying zero-based indexes of the columns of the input table that
+            contain projected partitioning expressions. This class will inspect these values for
+            each pair of consecutive input rows. When they change, this indicates the boundary
+            between two partitions, and we will invoke the 'terminate' method on the UDTF class
+            instance and then destroy it and create a new one to implement the desired partitioning
+            semantics.

Review Comment:
   I think these should be under `__init__`?
   ```
   def __init__(self, ..):
       """
       docstring here
       """
   ```



##########
python/pyspark/worker.py:
##########
@@ -573,9 +574,73 @@ def read_udtf(pickleSer, infile, eval_type):
             f"The return type of a UDTF must be a struct type, but got {type(return_type)}."
         )
 
+    class UDTFWithPartitions:
+        """
+        This implements the logic of a UDTF that accepts an input TABLE argument with one or more
+        PARTITION BY expressions.
+
+        Parameters
+        ----------
+        create_udtf: function
+            Function to create a new instance of the UDTF to be invoked.
+        partition_child_indexes: list
+            List of integers identifying zero-based indexes of the columns of the input table that
+            contain projected partitioning expressions. This class will inspect these values for
+            each pair of consecutive input rows. When they change, this indicates the boundary
+            between two partitions, and we will invoke the 'terminate' method on the UDTF class
+            instance and then destroy it and create a new one to implement the desired partitioning
+            semantics.
+        """
+        def __init__(self, create_udtf, partition_child_indexes):
+            self._create_udtf = create_udtf
+            self._udtf = create_udtf()
+            self._prev_arguments = None
+            self._partition_child_indexes = partition_child_indexes
+
+        def eval(self, *args, **kwargs):
+            changed_partitions = self._check_partition_boundaries(
+                list(args) + list(kwargs.values()))
+            if changed_partitions:
+                if self._udtf.terminate is not None:
+                    result = self._udtf.terminate()
+                    if result is not None:
+                        for row in result:
+                            yield row
+                self._udtf = self._create_udtf()

Review Comment:
   note: we should probably add some metrics here on how many times this udtf instance is created. 



-- 
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-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org