Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[SPARK-28128][PYTHON][SQL] Pandas Grouped UDFs skip empty partitions #24926

Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
13 changes: 13 additions & 0 deletions python/pyspark/sql/tests/test_pandas_udf_grouped_agg.py
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
import unittest

from pyspark.rdd import PythonEvalType
from pyspark.sql import Row
from pyspark.sql.functions import array, explode, col, lit, mean, sum, \
udf, pandas_udf, PandasUDFType
from pyspark.sql.types import *
Expand Down Expand Up @@ -461,6 +462,18 @@ def test_register_vectorized_udf_basic(self):
expected = [1, 5]
self.assertEqual(actual, expected)

def test_grouped_with_empty_partition(self):
data = [Row(id=1, x=2), Row(id=1, x=3), Row(id=2, x=4)]
expected = [Row(id=1, sum=5), Row(id=2, x=4)]
num_parts = len(data) + 1
df = self.spark.createDataFrame(self.sc.parallelize(data, numSlices=num_parts))

f = pandas_udf(lambda x: x.sum(),
'int', PandasUDFType.GROUPED_AGG)

result = df.groupBy('id').agg(f(df['x']).alias('sum')).collect()
self.assertEqual(result, expected)


if __name__ == "__main__":
from pyspark.sql.tests.test_pandas_udf_grouped_agg import *
Expand Down
12 changes: 12 additions & 0 deletions python/pyspark/sql/tests/test_pandas_udf_grouped_map.py
Original file line number Diff line number Diff line change
Expand Up @@ -504,6 +504,18 @@ def test_mixed_scalar_udfs_followed_by_grouby_apply(self):

self.assertEquals(result.collect()[0]['sum'], 165)

def test_grouped_with_empty_partition(self):
data = [Row(id=1, x=2), Row(id=1, x=3), Row(id=2, x=4)]
expected = [Row(id=1, x=5), Row(id=1, x=5), Row(id=2, x=4)]
num_parts = len(data) + 1
df = self.spark.createDataFrame(self.sc.parallelize(data, numSlices=num_parts))

f = pandas_udf(lambda pdf: pdf.assign(x=pdf['x'].sum()),
'id long, x int', PandasUDFType.GROUPED_MAP)

result = df.groupBy('id').apply(f).collect()
self.assertEqual(result, expected)


if __name__ == "__main__":
from pyspark.sql.tests.test_pandas_udf_grouped_map import *
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -105,7 +105,8 @@ case class AggregateInPandasExec(
StructField(s"_$i", dt)
})

inputRDD.mapPartitionsInternal { iter =>
// Map grouped rows to ArrowPythonRunner results, Only execute if partition is not empty
inputRDD.mapPartitionsInternal { iter => if (iter.isEmpty) iter else {
val prunedProj = UnsafeProjection.create(allInputs, child.output)

val grouped = if (groupingExpressions.isEmpty) {
Expand Down Expand Up @@ -151,6 +152,6 @@ case class AggregateInPandasExec(
val joinedRow = joined(leftRow, aggOutputRow)
resultProj(joinedRow)
}
}
}}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -125,7 +125,8 @@ case class FlatMapGroupsInPandasExec(
val dedupAttributes = nonDupGroupingAttributes ++ dataAttributes
val dedupSchema = StructType.fromAttributes(dedupAttributes)

inputRDD.mapPartitionsInternal { iter =>
// Map grouped rows to ArrowPythonRunner results, Only execute if partition is not empty
inputRDD.mapPartitionsInternal { iter => if (iter.isEmpty) iter else {
val grouped = if (groupingAttributes.isEmpty) {
Iterator(iter)
} else {
Expand Down Expand Up @@ -156,6 +157,6 @@ case class FlatMapGroupsInPandasExec(
flattenedBatch.setNumRows(batch.numRows())
flattenedBatch.rowIterator.asScala
}.map(unsafeProj)
}
}}
}
}