Skip to content

Commit 7b9fe08

Browse files
committed
[SPARK-23261][PYSPARK][BACKPORT-2.3] Rename Pandas UDFs
This PR is to backport #20428 to Spark 2.3 without adding the changes regarding `GROUPED AGG PANDAS UDF` --- ## What changes were proposed in this pull request? Rename the public APIs and names of pandas udfs. - `PANDAS SCALAR UDF` -> `SCALAR PANDAS UDF` - `PANDAS GROUP MAP UDF` -> `GROUPED MAP PANDAS UDF` ## How was this patch tested? The existing tests Author: gatorsmile <gatorsmile@gmail.com> Closes #20439 from gatorsmile/backport2.3.
1 parent f4802dc commit 7b9fe08

File tree

13 files changed

+96
-96
lines changed

13 files changed

+96
-96
lines changed

core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -37,14 +37,14 @@ private[spark] object PythonEvalType {
3737

3838
val SQL_BATCHED_UDF = 100
3939

40-
val SQL_PANDAS_SCALAR_UDF = 200
41-
val SQL_PANDAS_GROUP_MAP_UDF = 201
40+
val SQL_SCALAR_PANDAS_UDF = 200
41+
val SQL_GROUPED_MAP_PANDAS_UDF = 201
4242

4343
def toString(pythonEvalType: Int): String = pythonEvalType match {
4444
case NON_UDF => "NON_UDF"
4545
case SQL_BATCHED_UDF => "SQL_BATCHED_UDF"
46-
case SQL_PANDAS_SCALAR_UDF => "SQL_PANDAS_SCALAR_UDF"
47-
case SQL_PANDAS_GROUP_MAP_UDF => "SQL_PANDAS_GROUP_MAP_UDF"
46+
case SQL_SCALAR_PANDAS_UDF => "SQL_SCALAR_PANDAS_UDF"
47+
case SQL_GROUPED_MAP_PANDAS_UDF => "SQL_GROUPED_MAP_PANDAS_UDF"
4848
}
4949
}
5050

docs/sql-programming-guide.md

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -1684,7 +1684,7 @@ Spark will fall back to create the DataFrame without Arrow.
16841684
Pandas UDFs are user defined functions that are executed by Spark using Arrow to transfer data and
16851685
Pandas to work with the data. A Pandas UDF is defined using the keyword `pandas_udf` as a decorator
16861686
or to wrap the function, no additional configuration is required. Currently, there are two types of
1687-
Pandas UDF: Scalar and Group Map.
1687+
Pandas UDF: Scalar and Grouped Map.
16881688

16891689
### Scalar
16901690

@@ -1702,8 +1702,8 @@ The following example shows how to create a scalar Pandas UDF that computes the
17021702
</div>
17031703
</div>
17041704

1705-
### Group Map
1706-
Group map Pandas UDFs are used with `groupBy().apply()` which implements the "split-apply-combine" pattern.
1705+
### Grouped Map
1706+
Grouped map Pandas UDFs are used with `groupBy().apply()` which implements the "split-apply-combine" pattern.
17071707
Split-apply-combine consists of three steps:
17081708
* Split the data into groups by using `DataFrame.groupBy`.
17091709
* Apply a function on each group. The input and output of the function are both `pandas.DataFrame`. The
@@ -1723,7 +1723,7 @@ The following example shows how to use `groupby().apply()` to subtract the mean
17231723

17241724
<div class="codetabs">
17251725
<div data-lang="python" markdown="1">
1726-
{% include_example group_map_pandas_udf python/sql/arrow.py %}
1726+
{% include_example grouped_map_pandas_udf python/sql/arrow.py %}
17271727
</div>
17281728
</div>
17291729

examples/src/main/python/sql/arrow.py

Lines changed: 6 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -86,15 +86,15 @@ def multiply_func(a, b):
8686
# $example off:scalar_pandas_udf$
8787

8888

89-
def group_map_pandas_udf_example(spark):
90-
# $example on:group_map_pandas_udf$
89+
def grouped_map_pandas_udf_example(spark):
90+
# $example on:grouped_map_pandas_udf$
9191
from pyspark.sql.functions import pandas_udf, PandasUDFType
9292

9393
df = spark.createDataFrame(
9494
[(1, 1.0), (1, 2.0), (2, 3.0), (2, 5.0), (2, 10.0)],
9595
("id", "v"))
9696

97-
@pandas_udf("id long, v double", PandasUDFType.GROUP_MAP)
97+
@pandas_udf("id long, v double", PandasUDFType.GROUPED_MAP)
9898
def substract_mean(pdf):
9999
# pdf is a pandas.DataFrame
100100
v = pdf.v
@@ -110,7 +110,7 @@ def substract_mean(pdf):
110110
# | 2|-1.0|
111111
# | 2| 4.0|
112112
# +---+----+
113-
# $example off:group_map_pandas_udf$
113+
# $example off:grouped_map_pandas_udf$
114114

115115

116116
if __name__ == "__main__":
@@ -123,7 +123,7 @@ def substract_mean(pdf):
123123
dataframe_with_arrow_example(spark)
124124
print("Running pandas_udf scalar example")
125125
scalar_pandas_udf_example(spark)
126-
print("Running pandas_udf group map example")
127-
group_map_pandas_udf_example(spark)
126+
print("Running pandas_udf grouped map example")
127+
grouped_map_pandas_udf_example(spark)
128128

129129
spark.stop()

python/pyspark/rdd.py

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -68,8 +68,8 @@ class PythonEvalType(object):
6868

6969
SQL_BATCHED_UDF = 100
7070

71-
SQL_PANDAS_SCALAR_UDF = 200
72-
SQL_PANDAS_GROUP_MAP_UDF = 201
71+
SQL_SCALAR_PANDAS_UDF = 200
72+
SQL_GROUPED_MAP_PANDAS_UDF = 201
7373

7474

7575
def portable_hash(x):

python/pyspark/sql/functions.py

Lines changed: 12 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -1737,8 +1737,8 @@ def translate(srcCol, matching, replace):
17371737
def create_map(*cols):
17381738
"""Creates a new map column.
17391739
1740-
:param cols: list of column names (string) or list of :class:`Column` expressions that grouped
1741-
as key-value pairs, e.g. (key1, value1, key2, value2, ...).
1740+
:param cols: list of column names (string) or list of :class:`Column` expressions that are
1741+
grouped as key-value pairs, e.g. (key1, value1, key2, value2, ...).
17421742
17431743
>>> df.select(create_map('name', 'age').alias("map")).collect()
17441744
[Row(map={u'Alice': 2}), Row(map={u'Bob': 5})]
@@ -2085,9 +2085,9 @@ def map_values(col):
20852085
class PandasUDFType(object):
20862086
"""Pandas UDF Types. See :meth:`pyspark.sql.functions.pandas_udf`.
20872087
"""
2088-
SCALAR = PythonEvalType.SQL_PANDAS_SCALAR_UDF
2088+
SCALAR = PythonEvalType.SQL_SCALAR_PANDAS_UDF
20892089

2090-
GROUP_MAP = PythonEvalType.SQL_PANDAS_GROUP_MAP_UDF
2090+
GROUPED_MAP = PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF
20912091

20922092

20932093
@since(1.3)
@@ -2191,20 +2191,20 @@ def pandas_udf(f=None, returnType=None, functionType=None):
21912191
Therefore, this can be used, for example, to ensure the length of each returned
21922192
`pandas.Series`, and can not be used as the column length.
21932193
2194-
2. GROUP_MAP
2194+
2. GROUPED_MAP
21952195
2196-
A group map UDF defines transformation: A `pandas.DataFrame` -> A `pandas.DataFrame`
2196+
A grouped map UDF defines transformation: A `pandas.DataFrame` -> A `pandas.DataFrame`
21972197
The returnType should be a :class:`StructType` describing the schema of the returned
21982198
`pandas.DataFrame`.
21992199
The length of the returned `pandas.DataFrame` can be arbitrary.
22002200
2201-
Group map UDFs are used with :meth:`pyspark.sql.GroupedData.apply`.
2201+
Grouped map UDFs are used with :meth:`pyspark.sql.GroupedData.apply`.
22022202
22032203
>>> from pyspark.sql.functions import pandas_udf, PandasUDFType
22042204
>>> df = spark.createDataFrame(
22052205
... [(1, 1.0), (1, 2.0), (2, 3.0), (2, 5.0), (2, 10.0)],
22062206
... ("id", "v")) # doctest: +SKIP
2207-
>>> @pandas_udf("id long, v double", PandasUDFType.GROUP_MAP) # doctest: +SKIP
2207+
>>> @pandas_udf("id long, v double", PandasUDFType.GROUPED_MAP) # doctest: +SKIP
22082208
... def normalize(pdf):
22092209
... v = pdf.v
22102210
... return pdf.assign(v=(v - v.mean()) / v.std())
@@ -2254,20 +2254,20 @@ def pandas_udf(f=None, returnType=None, functionType=None):
22542254
eval_type = returnType
22552255
else:
22562256
# @pandas_udf(dataType) or @pandas_udf(returnType=dataType)
2257-
eval_type = PythonEvalType.SQL_PANDAS_SCALAR_UDF
2257+
eval_type = PythonEvalType.SQL_SCALAR_PANDAS_UDF
22582258
else:
22592259
return_type = returnType
22602260

22612261
if functionType is not None:
22622262
eval_type = functionType
22632263
else:
2264-
eval_type = PythonEvalType.SQL_PANDAS_SCALAR_UDF
2264+
eval_type = PythonEvalType.SQL_SCALAR_PANDAS_UDF
22652265

22662266
if return_type is None:
22672267
raise ValueError("Invalid returnType: returnType can not be None")
22682268

2269-
if eval_type not in [PythonEvalType.SQL_PANDAS_SCALAR_UDF,
2270-
PythonEvalType.SQL_PANDAS_GROUP_MAP_UDF]:
2269+
if eval_type not in [PythonEvalType.SQL_SCALAR_PANDAS_UDF,
2270+
PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF]:
22712271
raise ValueError("Invalid functionType: "
22722272
"functionType must be one the values from PandasUDFType")
22732273

python/pyspark/sql/group.py

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -212,14 +212,14 @@ def apply(self, udf):
212212
This function does not support partial aggregation, and requires shuffling all the data in
213213
the :class:`DataFrame`.
214214
215-
:param udf: a group map user-defined function returned by
216-
:meth:`pyspark.sql.functions.pandas_udf`.
215+
:param udf: a grouped map user-defined function returned by
216+
:func:`pyspark.sql.functions.pandas_udf`.
217217
218218
>>> from pyspark.sql.functions import pandas_udf, PandasUDFType
219219
>>> df = spark.createDataFrame(
220220
... [(1, 1.0), (1, 2.0), (2, 3.0), (2, 5.0), (2, 10.0)],
221221
... ("id", "v"))
222-
>>> @pandas_udf("id long, v double", PandasUDFType.GROUP_MAP) # doctest: +SKIP
222+
>>> @pandas_udf("id long, v double", PandasUDFType.GROUPED_MAP) # doctest: +SKIP
223223
... def normalize(pdf):
224224
... v = pdf.v
225225
... return pdf.assign(v=(v - v.mean()) / v.std())
@@ -239,9 +239,9 @@ def apply(self, udf):
239239
"""
240240
# Columns are special because hasattr always return True
241241
if isinstance(udf, Column) or not hasattr(udf, 'func') \
242-
or udf.evalType != PythonEvalType.SQL_PANDAS_GROUP_MAP_UDF:
242+
or udf.evalType != PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF:
243243
raise ValueError("Invalid udf: the udf argument must be a pandas_udf of type "
244-
"GROUP_MAP.")
244+
"GROUPED_MAP.")
245245
df = self._df
246246
udf_column = udf(*[df[col] for col in df.columns])
247247
jdf = self._jgd.flatMapGroupsInPandas(udf_column._jc.expr())

0 commit comments

Comments
 (0)