-
Notifications
You must be signed in to change notification settings - Fork 28.1k
/
group.py
536 lines (436 loc) · 16.5 KB
/
group.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
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
#
# 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 Callable, List, Optional, TYPE_CHECKING, overload, Dict, Union, cast, Tuple
from py4j.java_gateway import JavaObject
from pyspark.sql.column import Column, _to_seq
from pyspark.sql.session import SparkSession
from pyspark.sql.dataframe import DataFrame
from pyspark.sql.pandas.group_ops import PandasGroupedOpsMixin
if TYPE_CHECKING:
from pyspark.sql._typing import LiteralType
__all__ = ["GroupedData"]
def dfapi(f: Callable) -> Callable:
def _api(self: "GroupedData") -> DataFrame:
name = f.__name__
jdf = getattr(self._jgd, name)()
return DataFrame(jdf, self.session)
_api.__name__ = f.__name__
_api.__doc__ = f.__doc__
return _api
def df_varargs_api(f: Callable) -> Callable:
def _api(self: "GroupedData", *cols: str) -> DataFrame:
name = f.__name__
jdf = getattr(self._jgd, name)(_to_seq(self.session._sc, cols))
return DataFrame(jdf, self.session)
_api.__name__ = f.__name__
_api.__doc__ = f.__doc__
return _api
class GroupedData(PandasGroupedOpsMixin):
"""
A set of methods for aggregations on a :class:`DataFrame`,
created by :func:`DataFrame.groupBy`.
.. versionadded:: 1.3.0
.. versionchanged:: 3.4.0
Supports Spark Connect.
"""
def __init__(self, jgd: JavaObject, df: DataFrame):
self._jgd = jgd
self._df = df
self.session: SparkSession = df.sparkSession
@overload
def agg(self, *exprs: Column) -> DataFrame:
...
@overload
def agg(self, __exprs: Dict[str, str]) -> DataFrame:
...
def agg(self, *exprs: Union[Column, Dict[str, str]]) -> DataFrame:
"""Compute aggregates and returns the result as a :class:`DataFrame`.
The available aggregate functions can be:
1. built-in aggregation functions, such as `avg`, `max`, `min`, `sum`, `count`
2. group aggregate pandas UDFs, created with :func:`pyspark.sql.functions.pandas_udf`
.. note:: There is no partial aggregation with group aggregate UDFs, i.e.,
a full shuffle is required. Also, all the data of a group will be loaded into
memory, so the user should be aware of the potential OOM risk if data is skewed
and certain groups are too large to fit in memory.
.. seealso:: :func:`pyspark.sql.functions.pandas_udf`
If ``exprs`` is a single :class:`dict` mapping from string to string, then the key
is the column to perform aggregation on, and the value is the aggregate function.
Alternatively, ``exprs`` can also be a list of aggregate :class:`Column` expressions.
.. versionadded:: 1.3.0
.. versionchanged:: 3.4.0
Supports Spark Connect.
Parameters
----------
exprs : dict
a dict mapping from column name (string) to aggregate functions (string),
or a list of :class:`Column`.
Notes
-----
Built-in aggregation functions and group aggregate pandas UDFs cannot be mixed
in a single call to this function.
Examples
--------
>>> from pyspark.sql import functions as F
>>> from pyspark.sql.functions import pandas_udf, PandasUDFType
>>> df = spark.createDataFrame(
... [(2, "Alice"), (3, "Alice"), (5, "Bob"), (10, "Bob")], ["age", "name"])
>>> df.show()
+---+-----+
|age| name|
+---+-----+
| 2|Alice|
| 3|Alice|
| 5| Bob|
| 10| Bob|
+---+-----+
Group-by name, and count each group.
>>> df.groupBy(df.name).agg({"*": "count"}).sort("name").show()
+-----+--------+
| name|count(1)|
+-----+--------+
|Alice| 2|
| Bob| 2|
+-----+--------+
Group-by name, and calculate the minimum age.
>>> df.groupBy(df.name).agg(F.min(df.age)).sort("name").show()
+-----+--------+
| name|min(age)|
+-----+--------+
|Alice| 2|
| Bob| 5|
+-----+--------+
Same as above but uses pandas UDF.
>>> @pandas_udf('int', PandasUDFType.GROUPED_AGG) # doctest: +SKIP
... def min_udf(v):
... return v.min()
...
>>> df.groupBy(df.name).agg(min_udf(df.age)).sort("name").show() # doctest: +SKIP
+-----+------------+
| name|min_udf(age)|
+-----+------------+
|Alice| 2|
| Bob| 5|
+-----+------------+
"""
assert exprs, "exprs should not be empty"
if len(exprs) == 1 and isinstance(exprs[0], dict):
jdf = self._jgd.agg(exprs[0])
else:
# Columns
assert all(isinstance(c, Column) for c in exprs), "all exprs should be Column"
exprs = cast(Tuple[Column, ...], exprs)
jdf = self._jgd.agg(exprs[0]._jc, _to_seq(self.session._sc, [c._jc for c in exprs[1:]]))
return DataFrame(jdf, self.session)
@dfapi
def count(self) -> DataFrame:
"""Counts the number of records for each group.
.. versionadded:: 1.3.0
.. versionchanged:: 3.4.0
Supports Spark Connect.
Examples
--------
>>> df = spark.createDataFrame(
... [(2, "Alice"), (3, "Alice"), (5, "Bob"), (10, "Bob")], ["age", "name"])
>>> df.show()
+---+-----+
|age| name|
+---+-----+
| 2|Alice|
| 3|Alice|
| 5| Bob|
| 10| Bob|
+---+-----+
Group-by name, and count each group.
>>> df.groupBy(df.name).count().sort("name").show()
+-----+-----+
| name|count|
+-----+-----+
|Alice| 2|
| Bob| 2|
+-----+-----+
"""
@df_varargs_api
def mean(self, *cols: str) -> DataFrame:
"""Computes average values for each numeric columns for each group.
:func:`mean` is an alias for :func:`avg`.
.. versionadded:: 1.3.0
.. versionchanged:: 3.4.0
Supports Spark Connect.
Parameters
----------
cols : str
column names. Non-numeric columns are ignored.
"""
@df_varargs_api
def avg(self, *cols: str) -> DataFrame:
"""Computes average values for each numeric columns for each group.
:func:`mean` is an alias for :func:`avg`.
.. versionadded:: 1.3.0
.. versionchanged:: 3.4.0
Supports Spark Connect.
Parameters
----------
cols : str
column names. Non-numeric columns are ignored.
Examples
--------
>>> df = spark.createDataFrame([
... (2, "Alice", 80), (3, "Alice", 100),
... (5, "Bob", 120), (10, "Bob", 140)], ["age", "name", "height"])
>>> df.show()
+---+-----+------+
|age| name|height|
+---+-----+------+
| 2|Alice| 80|
| 3|Alice| 100|
| 5| Bob| 120|
| 10| Bob| 140|
+---+-----+------+
Group-by name, and calculate the mean of the age in each group.
>>> df.groupBy("name").avg('age').sort("name").show()
+-----+--------+
| name|avg(age)|
+-----+--------+
|Alice| 2.5|
| Bob| 7.5|
+-----+--------+
Calculate the mean of the age and height in all data.
>>> df.groupBy().avg('age', 'height').show()
+--------+-----------+
|avg(age)|avg(height)|
+--------+-----------+
| 5.0| 110.0|
+--------+-----------+
"""
@df_varargs_api
def max(self, *cols: str) -> DataFrame:
"""Computes the max value for each numeric columns for each group.
.. versionadded:: 1.3.0
.. versionchanged:: 3.4.0
Supports Spark Connect.
Examples
--------
>>> df = spark.createDataFrame([
... (2, "Alice", 80), (3, "Alice", 100),
... (5, "Bob", 120), (10, "Bob", 140)], ["age", "name", "height"])
>>> df.show()
+---+-----+------+
|age| name|height|
+---+-----+------+
| 2|Alice| 80|
| 3|Alice| 100|
| 5| Bob| 120|
| 10| Bob| 140|
+---+-----+------+
Group-by name, and calculate the max of the age in each group.
>>> df.groupBy("name").max("age").sort("name").show()
+-----+--------+
| name|max(age)|
+-----+--------+
|Alice| 3|
| Bob| 10|
+-----+--------+
Calculate the max of the age and height in all data.
>>> df.groupBy().max("age", "height").show()
+--------+-----------+
|max(age)|max(height)|
+--------+-----------+
| 10| 140|
+--------+-----------+
"""
@df_varargs_api
def min(self, *cols: str) -> DataFrame:
"""Computes the min value for each numeric column for each group.
.. versionadded:: 1.3.0
.. versionchanged:: 3.4.0
Supports Spark Connect.
Parameters
----------
cols : str
column names. Non-numeric columns are ignored.
Examples
--------
>>> df = spark.createDataFrame([
... (2, "Alice", 80), (3, "Alice", 100),
... (5, "Bob", 120), (10, "Bob", 140)], ["age", "name", "height"])
>>> df.show()
+---+-----+------+
|age| name|height|
+---+-----+------+
| 2|Alice| 80|
| 3|Alice| 100|
| 5| Bob| 120|
| 10| Bob| 140|
+---+-----+------+
Group-by name, and calculate the min of the age in each group.
>>> df.groupBy("name").min("age").sort("name").show()
+-----+--------+
| name|min(age)|
+-----+--------+
|Alice| 2|
| Bob| 5|
+-----+--------+
Calculate the min of the age and height in all data.
>>> df.groupBy().min("age", "height").show()
+--------+-----------+
|min(age)|min(height)|
+--------+-----------+
| 2| 80|
+--------+-----------+
"""
@df_varargs_api
def sum(self, *cols: str) -> DataFrame:
"""Computes the sum for each numeric columns for each group.
.. versionadded:: 1.3.0
.. versionchanged:: 3.4.0
Supports Spark Connect.
Parameters
----------
cols : str
column names. Non-numeric columns are ignored.
Examples
--------
>>> df = spark.createDataFrame([
... (2, "Alice", 80), (3, "Alice", 100),
... (5, "Bob", 120), (10, "Bob", 140)], ["age", "name", "height"])
>>> df.show()
+---+-----+------+
|age| name|height|
+---+-----+------+
| 2|Alice| 80|
| 3|Alice| 100|
| 5| Bob| 120|
| 10| Bob| 140|
+---+-----+------+
Group-by name, and calculate the sum of the age in each group.
>>> df.groupBy("name").sum("age").sort("name").show()
+-----+--------+
| name|sum(age)|
+-----+--------+
|Alice| 5|
| Bob| 15|
+-----+--------+
Calculate the sum of the age and height in all data.
>>> df.groupBy().sum("age", "height").show()
+--------+-----------+
|sum(age)|sum(height)|
+--------+-----------+
| 20| 440|
+--------+-----------+
"""
# TODO(SPARK-41746): SparkSession.createDataFrame does not support nested datatypes
def pivot(self, pivot_col: str, values: Optional[List["LiteralType"]] = None) -> "GroupedData":
"""
Pivots a column of the current :class:`DataFrame` and perform the specified aggregation.
There are two versions of the pivot function: one that requires the caller
to specify the list of distinct values to pivot on, and one that does not.
The latter is more concise but less efficient,
because Spark needs to first compute the list of distinct values internally.
.. versionadded:: 1.6.0
.. versionchanged:: 3.4.0
Supports Spark Connect.
Parameters
----------
pivot_col : str
Name of the column to pivot.
values : list, optional
List of values that will be translated to columns in the output DataFrame.
Examples
--------
>>> from pyspark.sql import Row
>>> df1 = spark.createDataFrame([
... Row(course="dotNET", year=2012, earnings=10000),
... Row(course="Java", year=2012, earnings=20000),
... Row(course="dotNET", year=2012, earnings=5000),
... Row(course="dotNET", year=2013, earnings=48000),
... Row(course="Java", year=2013, earnings=30000),
... ])
>>> df1.show()
+------+----+--------+
|course|year|earnings|
+------+----+--------+
|dotNET|2012| 10000|
| Java|2012| 20000|
|dotNET|2012| 5000|
|dotNET|2013| 48000|
| Java|2013| 30000|
+------+----+--------+
>>> df2 = spark.createDataFrame([
... Row(training="expert", sales=Row(course="dotNET", year=2012, earnings=10000)),
... Row(training="junior", sales=Row(course="Java", year=2012, earnings=20000)),
... Row(training="expert", sales=Row(course="dotNET", year=2012, earnings=5000)),
... Row(training="junior", sales=Row(course="dotNET", year=2013, earnings=48000)),
... Row(training="expert", sales=Row(course="Java", year=2013, earnings=30000)),
... ]) # doctest: +SKIP
>>> df2.show() # doctest: +SKIP
+--------+--------------------+
|training| sales|
+--------+--------------------+
| expert|{dotNET, 2012, 10...|
| junior| {Java, 2012, 20000}|
| expert|{dotNET, 2012, 5000}|
| junior|{dotNET, 2013, 48...|
| expert| {Java, 2013, 30000}|
+--------+--------------------+
Compute the sum of earnings for each year by course with each course as a separate column
>>> df1.groupBy("year").pivot("course", ["dotNET", "Java"]).sum("earnings").show()
+----+------+-----+
|year|dotNET| Java|
+----+------+-----+
|2012| 15000|20000|
|2013| 48000|30000|
+----+------+-----+
Or without specifying column values (less efficient)
>>> df1.groupBy("year").pivot("course").sum("earnings").show()
+----+-----+------+
|year| Java|dotNET|
+----+-----+------+
|2012|20000| 15000|
|2013|30000| 48000|
+----+-----+------+
>>> df2.groupBy("sales.year").pivot("sales.course").sum("sales.earnings").show()
... # doctest: +SKIP
+----+-----+------+
|year| Java|dotNET|
+----+-----+------+
|2012|20000| 15000|
|2013|30000| 48000|
+----+-----+------+
"""
if values is None:
jgd = self._jgd.pivot(pivot_col)
else:
jgd = self._jgd.pivot(pivot_col, values)
return GroupedData(jgd, self._df)
def _test() -> None:
import doctest
from pyspark.sql import SparkSession
import pyspark.sql.group
globs = pyspark.sql.group.__dict__.copy()
spark = SparkSession.builder.master("local[4]").appName("sql.group tests").getOrCreate()
globs["spark"] = spark
(failure_count, test_count) = doctest.testmod(
pyspark.sql.group,
globs=globs,
optionflags=doctest.ELLIPSIS | doctest.NORMALIZE_WHITESPACE | doctest.REPORT_NDIFF,
)
spark.stop()
if failure_count:
sys.exit(-1)
if __name__ == "__main__":
_test()