/
groupby.py
3413 lines (2932 loc) · 115 KB
/
groupby.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
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
from __future__ import annotations
import collections
import itertools as it
import operator
import uuid
import warnings
from functools import partial, wraps
from numbers import Integral
import numpy as np
import pandas as pd
from dask.base import is_dask_collection, tokenize
from dask.core import flatten
from dask.dataframe._compat import (
PANDAS_GE_140,
PANDAS_GE_150,
PANDAS_GE_200,
PANDAS_GE_210,
PANDAS_GE_220,
PANDAS_GE_300,
check_groupby_axis_deprecation,
check_numeric_only_deprecation,
check_observed_deprecation,
)
from dask.dataframe.core import (
GROUP_KEYS_DEFAULT,
DataFrame,
Series,
_convert_to_numeric,
_determine_split_out_shuffle,
_extract_meta,
_Frame,
aca,
map_partitions,
new_dd_object,
split_out_on_index,
)
from dask.dataframe.dispatch import grouper_dispatch
from dask.dataframe.methods import concat, drop_columns
from dask.dataframe.utils import (
get_numeric_only_kwargs,
insert_meta_param_description,
is_dataframe_like,
is_index_like,
is_series_like,
make_meta,
raise_on_meta_error,
)
from dask.highlevelgraph import HighLevelGraph
from dask.typing import no_default
from dask.utils import (
M,
_deprecated,
_deprecated_kwarg,
derived_from,
funcname,
itemgetter,
)
if PANDAS_GE_140:
from pandas.core.apply import reconstruct_func, validate_func_kwargs
# #############################################
#
# GroupBy implementation notes
#
# Dask groupby supports reductions, i.e., mean, sum and alike, and apply. The
# former do not shuffle the data and are efficiently implemented as tree
# reductions. The latter is implemented by shuffling the underlying partitions
# such that all items of a group can be found in the same partition.
#
# The argument to ``.groupby`` (``by``), can be a ``str``, ``dd.DataFrame``,
# ``dd.Series``, or a list thereof. In operations on the grouped object, the
# divisions of the the grouped object and the items of ``by`` have to align.
# Currently, there is no support to shuffle the ``by`` values as part of the
# groupby operation. Therefore, the alignment has to be guaranteed by the
# caller.
#
# To operate on matching partitions, most groupby operations exploit the
# corresponding support in ``apply_concat_apply``. Specifically, this function
# operates on matching partitions of frame-like objects passed as varargs.
#
# After the initial chunk step, ``by``` is implicitly passed along to
# subsequent operations as the index of the partitions. Groupby operations on
# the individual partitions can then access ``by`` via the ``levels``
# parameter of the ``groupby`` function. The correct argument is determined by
# the ``_determine_levels`` function.
#
# To minimize overhead, any ``by`` that is a series contained within the
# dataframe is passed as a column key. This transformation is implemented as
# ``_normalize_by``.
#
# #############################################
NUMERIC_ONLY_NOT_IMPLEMENTED = [
"mean",
"std",
"var",
]
def _determine_levels(by):
"""Determine the correct levels argument to groupby."""
if isinstance(by, (tuple, list)) and len(by) > 1:
return list(range(len(by)))
else:
return 0
def _normalize_by(df, by):
"""Replace series with column names wherever possible."""
if not isinstance(df, DataFrame):
return by
elif isinstance(by, list):
return [_normalize_by(df, col) for col in by]
elif is_series_like(by) and by.name in df.columns and by._name == df[by.name]._name:
return by.name
elif (
isinstance(by, DataFrame)
and set(by.columns).issubset(df.columns)
and by._name == df[by.columns]._name
):
return list(by.columns)
else:
return by
def _maybe_slice(grouped, columns):
"""
Slice columns if grouped is pd.DataFrameGroupBy
"""
# FIXME: update with better groupby object detection (i.e.: ngroups, get_group)
if "groupby" in type(grouped).__name__.lower():
if columns is not None:
if isinstance(columns, (tuple, list, set, pd.Index)):
columns = list(columns)
return grouped[columns]
return grouped
def _is_aligned(df, by):
"""Check if ``df`` and ``by`` have aligned indices"""
if is_series_like(by) or is_dataframe_like(by):
return df.index.equals(by.index)
elif isinstance(by, (list, tuple)):
return all(_is_aligned(df, i) for i in by)
else:
return True
def _groupby_raise_unaligned(df, convert_by_to_list=True, **kwargs):
"""Groupby, but raise if df and `by` key are unaligned.
Pandas supports grouping by a column that doesn't align with the input
frame/series/index. However, the reindexing does not seem to be
threadsafe, and can result in incorrect results. Since grouping by an
unaligned key is generally a bad idea, we just error loudly in dask.
For more information see pandas GH issue #15244 and Dask GH issue #1876."""
by = kwargs.get("by", None)
if by is not None and not _is_aligned(df, by):
msg = (
"Grouping by an unaligned column is unsafe and unsupported.\n"
"This can be caused by filtering only one of the object or\n"
"grouping key. For example, the following works in pandas,\n"
"but not in dask:\n"
"\n"
"df[df.foo < 0].groupby(df.bar)\n"
"\n"
"This can be avoided by either filtering beforehand, or\n"
"passing in the name of the column instead:\n"
"\n"
"df2 = df[df.foo < 0]\n"
"df2.groupby(df2.bar)\n"
"# or\n"
"df[df.foo < 0].groupby('bar')\n"
"\n"
"For more information see dask GH issue #1876."
)
raise ValueError(msg)
elif by is not None and len(by) and convert_by_to_list:
# since we're coming through apply, `by` will be a tuple.
# Pandas treats tuples as a single key, and lists as multiple keys
# We want multiple keys
if isinstance(by, str):
by = [by]
kwargs.update(by=list(by))
with check_observed_deprecation():
return df.groupby(**kwargs)
def _groupby_slice_apply(
df,
grouper,
key,
func,
*args,
group_keys=GROUP_KEYS_DEFAULT,
dropna=None,
observed=None,
**kwargs,
):
# No need to use raise if unaligned here - this is only called after
# shuffling, which makes everything aligned already
dropna = {"dropna": dropna} if dropna is not None else {}
observed = {"observed": observed} if observed is not None else {}
g = df.groupby(grouper, group_keys=group_keys, **observed, **dropna)
if key:
g = g[key]
return g.apply(func, *args, **kwargs)
def _groupby_slice_transform(
df,
grouper,
key,
func,
*args,
group_keys=GROUP_KEYS_DEFAULT,
dropna=None,
observed=None,
**kwargs,
):
# No need to use raise if unaligned here - this is only called after
# shuffling, which makes everything aligned already
dropna = {"dropna": dropna} if dropna is not None else {}
observed = {"observed": observed} if observed is not None else {}
g = df.groupby(grouper, group_keys=group_keys, **observed, **dropna)
if key:
g = g[key]
# Cannot call transform on an empty dataframe
if len(df) == 0:
return g.apply(func, *args, **kwargs)
return g.transform(func, *args, **kwargs)
def _groupby_slice_shift(
df,
grouper,
key,
shuffled,
group_keys=GROUP_KEYS_DEFAULT,
dropna=None,
observed=None,
**kwargs,
):
# No need to use raise if unaligned here - this is only called after
# shuffling, which makes everything aligned already
dropna = {"dropna": dropna} if dropna is not None else {}
observed = {"observed": observed} if observed is not None else {}
if shuffled:
df = df.sort_index()
g = df.groupby(grouper, group_keys=group_keys, **observed, **dropna)
if key:
g = g[key]
with check_groupby_axis_deprecation():
result = g.shift(**kwargs)
return result
def _groupby_get_group(df, by_key, get_key, columns):
# SeriesGroupBy may pass df which includes group key
grouped = _groupby_raise_unaligned(df, by=by_key, convert_by_to_list=False)
try:
if is_dataframe_like(df):
grouped = grouped[columns]
return grouped.get_group(get_key)
except KeyError:
# to create empty DataFrame/Series, which has the same
# dtype as the original
if is_dataframe_like(df):
# may be SeriesGroupBy
df = df[columns]
return df.iloc[0:0]
def numeric_only_deprecate_default(func):
"""Decorator for methods that should warn when numeric_only is default"""
@wraps(func)
def wrapper(self, *args, **kwargs):
if isinstance(self, DataFrameGroupBy):
numeric_only = kwargs.get("numeric_only", no_default)
# Prior to `pandas=1.5`, `numeric_only` support wasn't uniformly supported
# in pandas. We don't support `numeric_only=False` in this case.
if not PANDAS_GE_150 and numeric_only is False:
raise NotImplementedError(
"'numeric_only=False' is not implemented in Dask."
)
if PANDAS_GE_150 and not PANDAS_GE_200 and not self._all_numeric():
if numeric_only is no_default:
warnings.warn(
"The default value of numeric_only will be changed to False in "
"the future when using dask with pandas 2.0",
FutureWarning,
)
elif numeric_only is False and funcname(func) in ("sum", "prod"):
warnings.warn(
"Dropping invalid columns is deprecated. In a future version, a TypeError will be raised. "
f"Before calling .{funcname(func)}, select only columns which should be valid for the function",
FutureWarning,
)
return func(self, *args, **kwargs)
return wrapper
def numeric_only_not_implemented(func):
"""Decorator for methods that can't handle numeric_only=False"""
@wraps(func)
def wrapper(self, *args, **kwargs):
if isinstance(self, DataFrameGroupBy):
maybe_raise = not (
func.__name__ == "agg"
and len(args) > 0
and args[0] not in NUMERIC_ONLY_NOT_IMPLEMENTED
)
if maybe_raise:
numeric_only = kwargs.get("numeric_only", no_default)
# Prior to `pandas=1.5`, `numeric_only` support wasn't uniformly supported
# in pandas. We don't support `numeric_only=False` in this case.
if not PANDAS_GE_150 and numeric_only is False:
raise NotImplementedError(
"'numeric_only=False' is not implemented in Dask."
)
if not self._all_numeric():
if numeric_only is False or (
PANDAS_GE_200 and numeric_only is no_default
):
raise NotImplementedError(
"'numeric_only=False' is not implemented in Dask."
)
if (
PANDAS_GE_150
and not PANDAS_GE_200
and numeric_only is no_default
):
warnings.warn(
"The default value of numeric_only will be changed to False "
"in the future when using dask with pandas 2.0",
FutureWarning,
)
return func(self, *args, **kwargs)
return wrapper
###############################################################
# Aggregation
###############################################################
class Aggregation:
"""User defined groupby-aggregation.
This class allows users to define their own custom aggregation in terms of
operations on Pandas dataframes in a map-reduce style. You need to specify
what operation to do on each chunk of data, how to combine those chunks of
data together, and then how to finalize the result.
See :ref:`dataframe.groupby.aggregate` for more.
Parameters
----------
name : str
the name of the aggregation. It should be unique, since intermediate
result will be identified by this name.
chunk : callable
a function that will be called with the grouped column of each
partition. It can either return a single series or a tuple of series.
The index has to be equal to the groups.
agg : callable
a function that will be called to aggregate the results of each chunk.
Again the argument(s) will be grouped series. If ``chunk`` returned a
tuple, ``agg`` will be called with all of them as individual positional
arguments.
finalize : callable
an optional finalizer that will be called with the results from the
aggregation.
Examples
--------
We could implement ``sum`` as follows:
>>> custom_sum = dd.Aggregation(
... name='custom_sum',
... chunk=lambda s: s.sum(),
... agg=lambda s0: s0.sum()
... ) # doctest: +SKIP
>>> df.groupby('g').agg(custom_sum) # doctest: +SKIP
We can implement ``mean`` as follows:
>>> custom_mean = dd.Aggregation(
... name='custom_mean',
... chunk=lambda s: (s.count(), s.sum()),
... agg=lambda count, sum: (count.sum(), sum.sum()),
... finalize=lambda count, sum: sum / count,
... ) # doctest: +SKIP
>>> df.groupby('g').agg(custom_mean) # doctest: +SKIP
Though of course, both of these are built-in and so you don't need to
implement them yourself.
"""
def __init__(self, name, chunk, agg, finalize=None):
self.chunk = chunk
self.agg = agg
self.finalize = finalize
self.__name__ = name
def _groupby_aggregate(
df, aggfunc=None, levels=None, dropna=None, sort=False, observed=None, **kwargs
):
dropna = {"dropna": dropna} if dropna is not None else {}
observed = {"observed": observed} if observed is not None else {}
with check_observed_deprecation():
grouped = df.groupby(level=levels, sort=sort, **observed, **dropna)
# we emit a warning earlier in stack about default numeric_only being deprecated,
# so there's no need to propagate the warning that pandas emits as well
with check_numeric_only_deprecation():
return aggfunc(grouped, **kwargs)
def _groupby_aggregate_spec(
df, spec, levels=None, dropna=None, sort=False, observed=None, **kwargs
):
"""
A simpler version of _groupby_aggregate that just calls ``aggregate`` using
the user-provided spec.
"""
dropna = {"dropna": dropna} if dropna is not None else {}
observed = {"observed": observed} if observed is not None else {}
return df.groupby(level=levels, sort=sort, **observed, **dropna).aggregate(
spec, **kwargs
)
def _non_agg_chunk(df, *by, key, dropna=None, observed=None, **kwargs):
"""
A non-aggregation agg function. This simulates the behavior of an initial
partitionwise aggregation, but doesn't actually aggregate or throw away
any data.
"""
if is_series_like(df):
# Handle a series-like groupby. `by` could be columns that are not the series,
# but are like-indexed, so we handle that case by temporarily converting to
# a dataframe, then setting the index.
result = df.to_frame().set_index(by[0] if len(by) == 1 else list(by))[df.name]
else:
# Handle a frame-like groupby.
result = df.set_index(list(by))
if isinstance(key, (tuple, list, set, pd.Index)):
key = list(key)
result = result[key]
# If observed is False, we have to check for categorical indices and possibly enrich
# them with unobserved values. This function is intended as an initial partitionwise
# aggregation, so you might expect that we could enrich the frame with unobserved
# values at the end. However, if you have multiple output partitions, that results
# in duplicated unobserved values in each partition. So we have to do this step
# at the start before any shuffling occurs so that we can consolidate all of the
# unobserved values in a single partition.
if observed is False:
has_categoricals = False
# Search for categorical indices and get new index objects that have all the
# categories in them.
if isinstance(result.index, pd.CategoricalIndex):
has_categoricals = True
full_index = result.index.categories.copy().rename(result.index.name)
elif isinstance(result.index, pd.MultiIndex) and any(
isinstance(level, pd.CategoricalIndex) for level in result.index.levels
):
has_categoricals = True
full_index = pd.MultiIndex.from_product(
result.index.levels, names=result.index.names
)
if has_categoricals:
# If we found any categoricals, append unobserved values to the end of the
# frame.
new_cats = full_index[~full_index.isin(result.index)]
empty_data = {
c: pd.Series(index=new_cats, dtype=result[c].dtype)
for c in result.columns
}
empty = pd.DataFrame(empty_data)
result = pd.concat([result, empty])
return result
def _apply_chunk(df, *by, dropna=None, observed=None, **kwargs):
func = kwargs.pop("chunk")
columns = kwargs.pop("columns")
dropna = {"dropna": dropna} if dropna is not None else {}
observed = {"observed": observed} if observed is not None else {}
g = _groupby_raise_unaligned(df, by=by, **observed, **dropna)
if is_series_like(df) or columns is None:
return func(g, **kwargs)
else:
if isinstance(columns, (tuple, list, set, pd.Index)):
columns = list(columns)
return func(g[columns], **kwargs)
def _var_chunk(df, *by, numeric_only=no_default, observed=False, dropna=True):
numeric_only_kwargs = get_numeric_only_kwargs(numeric_only)
if is_series_like(df):
df = df.to_frame()
df = df.copy()
g = _groupby_raise_unaligned(df, by=by, observed=observed, dropna=dropna)
with check_numeric_only_deprecation():
x = g.sum(**numeric_only_kwargs)
n = g[x.columns].count().rename(columns=lambda c: (c, "-count"))
cols = x.columns
df[cols] = df[cols] ** 2
g2 = _groupby_raise_unaligned(df, by=by, observed=observed, dropna=dropna)
with check_numeric_only_deprecation():
x2 = g2.sum(**numeric_only_kwargs).rename(columns=lambda c: (c, "-x2"))
return concat([x, x2, n], axis=1)
def _var_combine(g, levels, sort=False):
return g.groupby(level=levels, sort=sort).sum()
def _var_agg(
g, levels, ddof, sort=False, numeric_only=no_default, observed=False, dropna=True
):
numeric_only_kwargs = get_numeric_only_kwargs(numeric_only)
g = g.groupby(level=levels, sort=sort, observed=observed, dropna=dropna).sum(
**numeric_only_kwargs
)
nc = len(g.columns)
x = g[g.columns[: nc // 3]]
# chunks columns are tuples (value, name), so we just keep the value part
x2 = g[g.columns[nc // 3 : 2 * nc // 3]].rename(columns=lambda c: c[0])
n = g[g.columns[-nc // 3 :]].rename(columns=lambda c: c[0])
# TODO: replace with _finalize_var?
result = x2 - x**2 / n
div = n - ddof
div[div < 0] = 0
result /= div
result[(n - ddof) == 0] = np.nan
assert is_dataframe_like(result)
result[result < 0] = 0 # avoid rounding errors that take us to zero
return result
def _cov_combine(g, levels):
return g
def _cov_finalizer(df, cols, std=False):
vals = []
num_elements = len(list(it.product(cols, repeat=2)))
num_cols = len(cols)
vals = list(range(num_elements))
col_idx_mapping = dict(zip(cols, range(num_cols)))
for i, j in it.combinations_with_replacement(df[cols].columns, 2):
x = col_idx_mapping[i]
y = col_idx_mapping[j]
idx = x + num_cols * y
mul_col = f"{i}{j}"
ni = df["%s-count" % i]
nj = df["%s-count" % j]
n = np.sqrt(ni * nj)
div = n - 1
div[div < 0] = 0
val = (df[mul_col] - df[i] * df[j] / n).values[0] / div.values[0]
if std:
ii = f"{i}{i}"
jj = f"{j}{j}"
std_val_i = (df[ii] - (df[i] ** 2) / ni).values[0] / div.values[0]
std_val_j = (df[jj] - (df[j] ** 2) / nj).values[0] / div.values[0]
sqrt_val = np.sqrt(std_val_i * std_val_j)
if sqrt_val == 0:
val = np.nan
else:
val = val / sqrt_val
vals[idx] = val
if i != j:
idx = num_cols * x + y
vals[idx] = val
level_1 = cols
index = pd.MultiIndex.from_product([level_1, level_1])
return pd.Series(vals, index=index)
def _mul_cols(df, cols):
"""Internal function to be used with apply to multiply
each column in a dataframe by every other column
a b c -> a*a, a*b, b*b, b*c, c*c
"""
_df = df.__class__()
for i, j in it.combinations_with_replacement(cols, 2):
col = f"{i}{j}"
_df[col] = df[i] * df[j]
# Fix index in a groupby().apply() context
# https://github.com/dask/dask/issues/8137
# https://github.com/pandas-dev/pandas/issues/43568
# Make sure index dtype is int (even if _df is empty)
# https://github.com/dask/dask/pull/9701
_df.index = np.zeros(len(_df), dtype=int)
return _df
def _cov_chunk(df, *by, numeric_only=no_default):
"""Covariance Chunk Logic
Parameters
----------
df : Pandas.DataFrame
std : bool, optional
When std=True we are calculating with Correlation
Returns
-------
tuple
Processed X, Multiplied Cols,
"""
numeric_only_kwargs = get_numeric_only_kwargs(numeric_only)
if is_series_like(df):
df = df.to_frame()
df = df.copy()
if numeric_only is False:
dt_df = df.select_dtypes(include=["datetime", "timedelta"])
for col in dt_df.columns:
df[col] = _convert_to_numeric(dt_df[col], True)
# mapping columns to str(numerical) values allows us to easily handle
# arbitrary column names (numbers, string, empty strings)
col_mapping = collections.OrderedDict()
for i, c in enumerate(df.columns):
col_mapping[c] = str(i)
df = df.rename(columns=col_mapping)
cols = df._get_numeric_data().columns
# when grouping by external series don't exclude columns
is_mask = any(is_series_like(s) for s in by)
if not is_mask:
by = [col_mapping[k] for k in by]
cols = cols.difference(pd.Index(by))
g = _groupby_raise_unaligned(df, by=by)
x = g.sum(**numeric_only_kwargs)
include_groups = {"include_groups": False} if PANDAS_GE_220 else {}
mul = g.apply(_mul_cols, cols=cols, **include_groups).reset_index(
level=-1, drop=True
)
n = g[x.columns].count().rename(columns=lambda c: f"{c}-count")
return (x, mul, n, col_mapping)
def _cov_agg(_t, levels, ddof, std=False, sort=False):
sums = []
muls = []
counts = []
# sometime we get a series back from concat combiner
t = list(_t)
cols = t[0][0].columns
for x, mul, n, col_mapping in t:
sums.append(x)
muls.append(mul)
counts.append(n)
col_mapping = col_mapping
total_sums = concat(sums).groupby(level=levels, sort=sort).sum()
total_muls = concat(muls).groupby(level=levels, sort=sort).sum()
total_counts = concat(counts).groupby(level=levels).sum()
result = (
concat([total_sums, total_muls, total_counts], axis=1)
.groupby(level=levels)
.apply(_cov_finalizer, cols=cols, std=std)
)
inv_col_mapping = {v: k for k, v in col_mapping.items()}
idx_vals = result.index.names
idx_mapping = list()
# when index is None we probably have selected a particular column
# df.groupby('a')[['b']].cov()
if len(idx_vals) == 1 and all(n is None for n in idx_vals):
idx_vals = list(inv_col_mapping.keys() - set(total_sums.columns))
for val in idx_vals:
idx_name = inv_col_mapping.get(val, val)
idx_mapping.append(idx_name)
if len(result.columns.levels[0]) < len(col_mapping):
# removing index from col_mapping (produces incorrect multiindexes)
try:
col_mapping.pop(idx_name)
except KeyError:
# when slicing the col_map will not have the index
pass
keys = list(col_mapping.keys())
for level in range(len(result.columns.levels)):
result.columns = result.columns.set_levels(keys, level=level)
result.index.set_names(idx_mapping, inplace=True)
# stacking can lead to a sorted index
if PANDAS_GE_300:
s_result = result.stack()
else:
s_result = result.stack(dropna=False)
assert is_dataframe_like(s_result)
return s_result
###############################################################
# nunique
###############################################################
def _nunique_df_chunk(df, *by, **kwargs):
name = kwargs.pop("name")
try:
# This is a lot faster but kind of a pain to implement when by
# has a boolean series in it.
return df.drop_duplicates(subset=list(by) + [name]).set_index(list(by))
except Exception:
pass
group_keys = {}
if PANDAS_GE_150:
group_keys["group_keys"] = True
g = _groupby_raise_unaligned(df, by=by, **group_keys)
if len(df) > 0:
grouped = g[name].unique().explode().to_frame()
else:
# Manually create empty version, since groupby-apply for empty frame
# results in df with no columns
grouped = g[[name]].nunique()
grouped = grouped.astype(df.dtypes[grouped.columns].to_dict())
return grouped
def _nunique_df_combine(df, levels, sort=False):
result = (
df.groupby(level=levels, sort=sort, observed=True)[df.columns[0]]
.unique()
.explode()
.to_frame()
)
return result
def _nunique_df_aggregate(df, levels, name, sort=False):
return df.groupby(level=levels, sort=sort, observed=True)[name].nunique()
def _nunique_series_chunk(df, *by, **_ignored_):
# convert series to data frame, then hand over to dataframe code path
assert is_series_like(df)
df = df.to_frame()
kwargs = dict(name=df.columns[0], levels=_determine_levels(by))
return _nunique_df_chunk(df, *by, **kwargs)
###############################################################
# Aggregate support
#
# Aggregate is implemented as:
#
# 1. group-by-aggregate all partitions into intermediate values
# 2. collect all partitions into a single partition
# 3. group-by-aggregate the result into intermediate values
# 4. transform all intermediate values into the result
#
# In Step 1 and 3 the dataframe is grouped on the same columns.
#
###############################################################
def _make_agg_id(func, column):
return f"{func!s}-{column!s}-{tokenize(func, column)}"
def _normalize_spec(spec, non_group_columns):
"""
Return a list of ``(result_column, func, input_column)`` tuples.
Spec can be
- a function
- a list of functions
- a dictionary that maps input-columns to functions
- a dictionary that maps input-columns to a lists of functions
- a dictionary that maps input-columns to a dictionaries that map
output-columns to functions.
The non-group columns are a list of all column names that are not used in
the groupby operation.
Usually, the result columns are mutli-level names, returned as tuples.
If only a single function is supplied or dictionary mapping columns
to single functions, simple names are returned as strings (see the first
two examples below).
Examples
--------
>>> _normalize_spec('mean', ['a', 'b', 'c'])
[('a', 'mean', 'a'), ('b', 'mean', 'b'), ('c', 'mean', 'c')]
>>> spec = collections.OrderedDict([('a', 'mean'), ('b', 'count')])
>>> _normalize_spec(spec, ['a', 'b', 'c'])
[('a', 'mean', 'a'), ('b', 'count', 'b')]
>>> _normalize_spec(['var', 'mean'], ['a', 'b', 'c'])
... # doctest: +NORMALIZE_WHITESPACE
[(('a', 'var'), 'var', 'a'), (('a', 'mean'), 'mean', 'a'), \
(('b', 'var'), 'var', 'b'), (('b', 'mean'), 'mean', 'b'), \
(('c', 'var'), 'var', 'c'), (('c', 'mean'), 'mean', 'c')]
>>> spec = collections.OrderedDict([('a', 'mean'), ('b', ['sum', 'count'])])
>>> _normalize_spec(spec, ['a', 'b', 'c'])
... # doctest: +NORMALIZE_WHITESPACE
[(('a', 'mean'), 'mean', 'a'), (('b', 'sum'), 'sum', 'b'), \
(('b', 'count'), 'count', 'b')]
>>> spec = collections.OrderedDict()
>>> spec['a'] = ['mean', 'size']
>>> spec['b'] = collections.OrderedDict([('e', 'count'), ('f', 'var')])
>>> _normalize_spec(spec, ['a', 'b', 'c'])
... # doctest: +NORMALIZE_WHITESPACE
[(('a', 'mean'), 'mean', 'a'), (('a', 'size'), 'size', 'a'), \
(('b', 'e'), 'count', 'b'), (('b', 'f'), 'var', 'b')]
"""
if not isinstance(spec, dict):
spec = collections.OrderedDict(zip(non_group_columns, it.repeat(spec)))
res = []
if isinstance(spec, dict):
for input_column, subspec in spec.items():
if isinstance(subspec, dict):
res.extend(
((input_column, result_column), func, input_column)
for result_column, func in subspec.items()
)
else:
if not isinstance(subspec, list):
subspec = [subspec]
res.extend(
((input_column, funcname(func)), func, input_column)
for func in subspec
)
else:
raise ValueError(f"unsupported agg spec of type {type(spec)}")
compounds = (list, tuple, dict)
use_flat_columns = not any(
isinstance(subspec, compounds) for subspec in spec.values()
)
if use_flat_columns:
res = [(input_col, func, input_col) for (_, func, input_col) in res]
return res
def _build_agg_args(spec):
"""
Create transformation functions for a normalized aggregate spec.
Parameters
----------
spec: a list of (result-column, aggregation-function, input-column) triples.
To work with all argument forms understood by pandas use
``_normalize_spec`` to normalize the argument before passing it on to
``_build_agg_args``.
Returns
-------
chunk_funcs: a list of (intermediate-column, function, keyword) triples
that are applied on grouped chunks of the initial dataframe.
agg_funcs: a list of (intermediate-column, functions, keyword) triples that
are applied on the grouped concatenation of the preprocessed chunks.
finalizers: a list of (result-column, function, keyword) triples that are
applied after the ``agg_funcs``. They are used to create final results
from intermediate representations.
"""
known_np_funcs = {
np.min: "min",
np.max: "max",
np.median: "median",
np.std: "std",
np.var: "var",
}
# check that there are no name conflicts for a single input column
by_name = {}
for _, func, input_column in spec:
key = funcname(known_np_funcs.get(func, func)), input_column
by_name.setdefault(key, []).append((func, input_column))
for funcs in by_name.values():
if len(funcs) != 1:
raise ValueError(f"conflicting aggregation functions: {funcs}")
chunks = {}
aggs = {}
finalizers = []
# a partial may contain some arguments, pass them down
# https://github.com/dask/dask/issues/9615
for result_column, func, input_column in spec:
func_args = ()
func_kwargs = {}
if isinstance(func, partial):
func_args, func_kwargs = func.args, func.keywords
if not isinstance(func, Aggregation):
func = funcname(known_np_funcs.get(func, func))
impls = _build_agg_args_single(
result_column, func, func_args, func_kwargs, input_column
)
# overwrite existing result-columns, generate intermediates only once
for spec in impls["chunk_funcs"]:
chunks[spec[0]] = spec
for spec in impls["aggregate_funcs"]:
aggs[spec[0]] = spec
finalizers.append(impls["finalizer"])
chunks = sorted(chunks.values())
aggs = sorted(aggs.values())
return chunks, aggs, finalizers
def _build_agg_args_single(result_column, func, func_args, func_kwargs, input_column):
simple_impl = {
"sum": (M.sum, M.sum),
"min": (M.min, M.min),
"max": (M.max, M.max),
"count": (M.count, M.sum),
"size": (M.size, M.sum),
"first": (M.first, M.first),
"last": (M.last, M.last),
"prod": (M.prod, M.prod),
"median": (
None,
M.median,
), # No chunk func for median, we can only take it when aggregating
}
if func in simple_impl.keys():
return _build_agg_args_simple(
result_column, func, input_column, simple_impl[func]
)
elif func == "var":
return _build_agg_args_var(
result_column, func, func_args, func_kwargs, input_column
)
elif func == "std":