-
Notifications
You must be signed in to change notification settings - Fork 28.3k
/
map_ops.py
217 lines (183 loc) · 7.97 KB
/
map_ops.py
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
#
# 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.
#
import sys
from typing import Union, TYPE_CHECKING
from pyspark.rdd import PythonEvalType
from pyspark.sql.types import StructType
if TYPE_CHECKING:
from pyspark.sql.dataframe import DataFrame
from pyspark.sql.pandas._typing import PandasMapIterFunction, ArrowMapIterFunction
class PandasMapOpsMixin:
"""
Min-in for pandas map operations. Currently, only :class:`DataFrame`
can use this class.
"""
def mapInPandas(
self, func: "PandasMapIterFunction", schema: Union[StructType, str], barrier: bool = False
) -> "DataFrame":
"""
Maps an iterator of batches in the current :class:`DataFrame` using a Python native
function that takes and outputs a pandas DataFrame, and returns the result as a
:class:`DataFrame`.
The function should take an iterator of `pandas.DataFrame`\\s and return
another iterator of `pandas.DataFrame`\\s. All columns are passed
together as an iterator of `pandas.DataFrame`\\s to the function and the
returned iterator of `pandas.DataFrame`\\s are combined as a :class:`DataFrame`.
Each `pandas.DataFrame` size can be controlled by
`spark.sql.execution.arrow.maxRecordsPerBatch`. The size of the function's input and
output can be different.
.. versionadded:: 3.0.0
.. versionchanged:: 3.4.0
Supports Spark Connect.
Parameters
----------
func : function
a Python native function that takes an iterator of `pandas.DataFrame`\\s, and
outputs an iterator of `pandas.DataFrame`\\s.
schema : :class:`pyspark.sql.types.DataType` or str
the return type of the `func` in PySpark. The value can be either a
:class:`pyspark.sql.types.DataType` object or a DDL-formatted type string.
barrier : bool, optional, default True
Use barrier mode execution.
.. versionchanged: 3.5.0
Added ``barrier`` argument.
Examples
--------
>>> from pyspark.sql.functions import pandas_udf
>>> df = spark.createDataFrame([(1, 21), (2, 30)], ("id", "age"))
>>> def filter_func(iterator):
... for pdf in iterator:
... yield pdf[pdf.id == 1]
>>> df.mapInPandas(filter_func, df.schema).show() # doctest: +SKIP
+---+---+
| id|age|
+---+---+
| 1| 21|
+---+---+
Set ``barrier`` to ``True`` to force the ``mapInPandas`` stage running in the
barrier mode, it ensures all Python workers in the stage will be
launched concurrently.
>>> df.mapInPandas(filter_func, df.schema, barrier=True).show() # doctest: +SKIP
+---+---+
| id|age|
+---+---+
| 1| 21|
+---+---+
Notes
-----
This API is experimental
See Also
--------
pyspark.sql.functions.pandas_udf
"""
from pyspark.sql import DataFrame
from pyspark.sql.pandas.functions import pandas_udf
assert isinstance(self, DataFrame)
# The usage of the pandas_udf is internal so type checking is disabled.
udf = pandas_udf(
func, returnType=schema, functionType=PythonEvalType.SQL_MAP_PANDAS_ITER_UDF
) # type: ignore[call-overload]
udf_column = udf(*[self[col] for col in self.columns])
jdf = self._jdf.mapInPandas(udf_column._jc.expr(), barrier)
return DataFrame(jdf, self.sparkSession)
def mapInArrow(
self, func: "ArrowMapIterFunction", schema: Union[StructType, str], barrier: bool = False
) -> "DataFrame":
"""
Maps an iterator of batches in the current :class:`DataFrame` using a Python native
function that takes and outputs a PyArrow's `RecordBatch`, and returns the result as a
:class:`DataFrame`.
The function should take an iterator of `pyarrow.RecordBatch`\\s and return
another iterator of `pyarrow.RecordBatch`\\s. All columns are passed
together as an iterator of `pyarrow.RecordBatch`\\s to the function and the
returned iterator of `pyarrow.RecordBatch`\\s are combined as a :class:`DataFrame`.
Each `pyarrow.RecordBatch` size can be controlled by
`spark.sql.execution.arrow.maxRecordsPerBatch`. The size of the function's input and
output can be different.
.. versionadded:: 3.3.0
Parameters
----------
func : function
a Python native function that takes an iterator of `pyarrow.RecordBatch`\\s, and
outputs an iterator of `pyarrow.RecordBatch`\\s.
schema : :class:`pyspark.sql.types.DataType` or str
the return type of the `func` in PySpark. The value can be either a
:class:`pyspark.sql.types.DataType` object or a DDL-formatted type string.
barrier : bool, optional, default True
Use barrier mode execution.
.. versionchanged: 3.5.0
Added ``barrier`` argument.
Examples
--------
>>> import pyarrow # doctest: +SKIP
>>> df = spark.createDataFrame([(1, 21), (2, 30)], ("id", "age"))
>>> def filter_func(iterator):
... for batch in iterator:
... pdf = batch.to_pandas()
... yield pyarrow.RecordBatch.from_pandas(pdf[pdf.id == 1])
>>> df.mapInArrow(filter_func, df.schema).show() # doctest: +SKIP
+---+---+
| id|age|
+---+---+
| 1| 21|
+---+---+
Set ``barrier`` to ``True`` to force the ``mapInArrow`` stage running in the
barrier mode, it ensures all Python workers in the stage will be
launched concurrently.
>>> df.mapInArrow(filter_func, df.schema, barrier=True).show() # doctest: +SKIP
+---+---+
| id|age|
+---+---+
| 1| 21|
+---+---+
Notes
-----
This API is unstable, and for developers.
See Also
--------
pyspark.sql.functions.pandas_udf
pyspark.sql.DataFrame.mapInPandas
"""
from pyspark.sql import DataFrame
from pyspark.sql.pandas.functions import pandas_udf
assert isinstance(self, DataFrame)
# The usage of the pandas_udf is internal so type checking is disabled.
udf = pandas_udf(
func, returnType=schema, functionType=PythonEvalType.SQL_MAP_ARROW_ITER_UDF
) # type: ignore[call-overload]
udf_column = udf(*[self[col] for col in self.columns])
jdf = self._jdf.pythonMapInArrow(udf_column._jc.expr(), barrier)
return DataFrame(jdf, self.sparkSession)
def _test() -> None:
import doctest
from pyspark.sql import SparkSession
import pyspark.sql.pandas.map_ops
globs = pyspark.sql.pandas.map_ops.__dict__.copy()
spark = (
SparkSession.builder.master("local[4]").appName("sql.pandas.map_ops tests").getOrCreate()
)
globs["spark"] = spark
(failure_count, test_count) = doctest.testmod(
pyspark.sql.pandas.map_ops,
globs=globs,
optionflags=doctest.ELLIPSIS | doctest.NORMALIZE_WHITESPACE | doctest.REPORT_NDIFF,
)
spark.stop()
if failure_count:
sys.exit(-1)
if __name__ == "__main__":
_test()