-
Notifications
You must be signed in to change notification settings - Fork 2
/
groupby.py
1220 lines (960 loc) · 43.1 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 absolute_import, division, print_function
import collections
import itertools as it
import operator
import warnings
import numpy as np
import pandas as pd
from .core import (DataFrame, Series, aca, map_partitions, merge,
new_dd_object, no_default, split_out_on_index)
from .methods import drop_columns
from .shuffle import shuffle
from .utils import make_meta, insert_meta_param_description, raise_on_meta_error
from ..base import tokenize
from ..utils import derived_from, M, funcname, itemgetter
# #############################################
#
# 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 partiitons
# such that all items of a group can be found in the same parititon.
#
# The argument to ``.groupby``, the index, 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 index have to align.
# Currently, there is no support to shuffle the index values as part of the
# groupby operation. Therefore, the alignment has to be guaranteed by the
# caller.
#
# To operate on matchings paritions, most groupby operations exploit the
# corresponding support in ``apply_concat_apply``. Specifically, this function
# operates on matching paritiotns of frame-like objects passed as varargs.
#
# After the inital chunk step, the passed index is implicitly passed along to
# subsequent operations as the index of the parittions. Groupby operations on
# the individual parttions can then access the index via the ``levels``
# parameter of the ``groupby`` function. The correct arguments is determined by
# the ``_determine_levels`` function.
#
# To minimize overhead, series in an index that were obtained by getitem on the
# object to group are not passed as series to the various operations, but as
# columnn keys. This transformation is implemented as ``_normalize_index``.
#
# #############################################
def _determine_levels(index):
"""Determine the correct levels argument to groupby.
"""
if isinstance(index, (tuple, list)) and len(index) > 1:
return list(range(len(index)))
else:
return 0
def _normalize_index(df, index):
"""Replace series with column names in an index wherever possible.
"""
if not isinstance(df, DataFrame):
return index
elif isinstance(index, list):
return [_normalize_index(df, col) for col in index]
elif (isinstance(index, Series) and index.name in df.columns and
index._name == df[index.name]._name):
return index.name
elif (isinstance(index, DataFrame) and
set(index.columns).issubset(df.columns) and
index._name == df[index.columns]._name):
return list(index.columns)
else:
return index
def _maybe_slice(grouped, columns):
"""
Slice columns if grouped is pd.DataFrameGroupBy
"""
if isinstance(grouped, pd.core.groupby.DataFrameGroupBy):
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 isinstance(by, (pd.Series, pd.DataFrame)):
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, **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 this causes doesn't 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 index 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):
# 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
kwargs.update(by=list(by))
return df.groupby(**kwargs)
def _groupby_slice_apply(df, grouper, key, func):
# No need to use raise if unaligned here - this is only called after
# shuffling, which makes everything aligned already
if isinstance(grouper, (pd.DataFrame, pd.Series, pd.Index)):
grouper = grouper.sort_values()
else:
try:
df = df.sort_values(grouper)
except KeyError: # this fails when the grouper includes the index
pass
g = df.groupby(grouper)
if key:
g = g[key]
return g.apply(func)
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)
if get_key in grouped.groups:
if isinstance(df, pd.DataFrame):
grouped = grouped[columns]
return grouped.get_group(get_key)
else:
# to create empty DataFrame/Series, which has the same
# dtype as the original
if isinstance(df, pd.DataFrame):
# may be SeriesGroupBy
df = df[columns]
return df.iloc[0:0]
###############################################################
# Aggregation
###############################################################
# Implementation detail: use class to make it easier to pass inside spec
class Aggregation(object):
"""A user defined aggregation.
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
--------
``sum`` can be implemented as::
custom_sum = dd.Aggregation('custom_sum', lambda s: s.sum(), lambda s0: s0.sum())
df.groupby('g').agg(custom_sum)
and ``mean`` can be implemented as::
custom_mean = dd.Aggregation(
'custom_mean',
lambda s: (s.count(), s.sum()),
lambda count, sum: (count.sum(), sum.sum()),
lambda count, sum: sum / count,
)
df.groupby('g').agg(custom_mean)
"""
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):
return aggfunc(df.groupby(level=levels, sort=False))
def _apply_chunk(df, *index, **kwargs):
func = kwargs.pop('chunk')
columns = kwargs.pop('columns')
g = _groupby_raise_unaligned(df, by=index)
if isinstance(df, pd.Series) or columns is None:
return func(g)
else:
if isinstance(columns, (tuple, list, set, pd.Index)):
columns = list(columns)
return func(g[columns])
def _var_chunk(df, *index):
if isinstance(df, pd.Series):
df = df.to_frame()
g = _groupby_raise_unaligned(df, by=index)
x = g.sum()
x2 = g.agg(lambda x: (x**2).sum()).rename(columns=lambda c: c + '-x2')
n = g.count().rename(columns=lambda c: c + '-count')
return pd.concat([x, x2, n], axis=1)
def _var_combine(g, levels):
return g.groupby(level=levels, sort=False).sum()
def _var_agg(g, levels, ddof):
g = g.groupby(level=levels, sort=False).sum()
nc = len(g.columns)
x = g[g.columns[:nc // 3]]
x2 = g[g.columns[nc // 3:2 * nc // 3]].rename(columns=lambda c: c[:-3])
n = g[g.columns[-nc // 3:]].rename(columns=lambda c: c[:-6])
# 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 isinstance(result, pd.DataFrame)
return result
###############################################################
# nunique
###############################################################
def _nunique_df_chunk(df, *index, **kwargs):
levels = kwargs.pop('levels')
name = kwargs.pop('name')
g = _groupby_raise_unaligned(df, by=index)
grouped = g[[name]].apply(pd.DataFrame.drop_duplicates)
# we set the index here to force a possibly duplicate index
# for our reduce step
if isinstance(levels, list):
grouped.index = pd.MultiIndex.from_arrays([
grouped.index.get_level_values(level=level) for level in levels
])
else:
grouped.index = grouped.index.get_level_values(level=levels)
return grouped
def _drop_duplicates_rename(df):
# Avoid duplicate index labels in a groupby().apply() context
# https://github.com/dask/dask/issues/3039
# https://github.com/pandas-dev/pandas/pull/18882
names = [None] * df.index.nlevels
return df.drop_duplicates().rename_axis(names, copy=False)
def _nunique_df_combine(df, levels):
result = df.groupby(level=levels,
sort=False).apply(_drop_duplicates_rename)
if isinstance(levels, list):
result.index = pd.MultiIndex.from_arrays([
result.index.get_level_values(level=level) for level in levels
])
else:
result.index = result.index.get_level_values(level=levels)
return result
def _nunique_df_aggregate(df, levels, name):
return df.groupby(level=levels, sort=False)[name].nunique()
def _nunique_series_chunk(df, *index, **_ignored_):
# convert series to data frame, then hand over to dataframe code path
assert isinstance(df, pd.Series)
df = df.to_frame()
kwargs = dict(name=df.columns[0], levels=_determine_levels(index))
return _nunique_df_chunk(df, *index, **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 '{!s}-{!s}-{}'.format(func, column, 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("unsupported agg spec of type {}".format(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 arugment forms understood by pandas use
``_normalize_spec`` to normalize the argment 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, keword) triples that
are applied on the grouped concatination 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'}
# 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('conflicting aggregation functions: {}'.format(funcs))
chunks = {}
aggs = {}
finalizers = []
for (result_column, func, input_column) in spec:
if not isinstance(func, Aggregation):
func = funcname(known_np_funcs.get(func, func))
impls = _build_agg_args_single(result_column, func, input_column)
# overwrite existing result-columns, generate intermediates only once
chunks.update((spec[0], spec) for spec in impls['chunk_funcs'])
aggs.update((spec[0], spec) for spec in impls['aggregate_funcs'])
finalizers.append(impls['finalizer'])
chunks = sorted(chunks.values())
aggs = sorted(aggs.values())
return chunks, aggs, finalizers
def _build_agg_args_single(result_column, func, 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),
}
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, input_column)
elif func == 'std':
return _build_agg_args_std(result_column, func, input_column)
elif func == 'mean':
return _build_agg_args_mean(result_column, func, input_column)
elif isinstance(func, Aggregation):
return _build_agg_args_custom(result_column, func, input_column)
else:
raise ValueError("unknown aggregate {}".format(func))
def _build_agg_args_simple(result_column, func, input_column, impl_pair):
intermediate = _make_agg_id(func, input_column)
chunk_impl, agg_impl = impl_pair
return dict(
chunk_funcs=[(intermediate, _apply_func_to_column,
dict(column=input_column, func=chunk_impl))],
aggregate_funcs=[(intermediate, _apply_func_to_column,
dict(column=intermediate, func=agg_impl))],
finalizer=(result_column, itemgetter(intermediate), dict()),
)
def _build_agg_args_var(result_column, func, input_column):
int_sum = _make_agg_id('sum', input_column)
int_sum2 = _make_agg_id('sum2', input_column)
int_count = _make_agg_id('count', input_column)
return dict(
chunk_funcs=[
(int_sum, _apply_func_to_column,
dict(column=input_column, func=M.sum)),
(int_count, _apply_func_to_column,
dict(column=input_column, func=M.count)),
(int_sum2, _compute_sum_of_squares,
dict(column=input_column)),
],
aggregate_funcs=[
(col, _apply_func_to_column, dict(column=col, func=M.sum))
for col in (int_sum, int_count, int_sum2)
],
finalizer=(result_column, _finalize_var,
dict(sum_column=int_sum, count_column=int_count,
sum2_column=int_sum2)),
)
def _build_agg_args_std(result_column, func, input_column):
impls = _build_agg_args_var(result_column, func, input_column)
result_column, _, kwargs = impls['finalizer']
impls['finalizer'] = (result_column, _finalize_std, kwargs)
return impls
def _build_agg_args_mean(result_column, func, input_column):
int_sum = _make_agg_id('sum', input_column)
int_count = _make_agg_id('count', input_column)
return dict(
chunk_funcs=[
(int_sum, _apply_func_to_column,
dict(column=input_column, func=M.sum)),
(int_count, _apply_func_to_column,
dict(column=input_column, func=M.count)),
],
aggregate_funcs=[
(col, _apply_func_to_column, dict(column=col, func=M.sum))
for col in (int_sum, int_count)
],
finalizer=(result_column, _finalize_mean,
dict(sum_column=int_sum, count_column=int_count)),
)
def _build_agg_args_custom(result_column, func, input_column):
col = _make_agg_id(funcname(func), input_column)
if func.finalize is None:
finalizer = (result_column, operator.itemgetter(col), dict())
else:
finalizer = (
result_column, _apply_func_to_columns,
dict(func=func.finalize, prefix=col)
)
return dict(
chunk_funcs=[
(col, _apply_func_to_column,
dict(func=func.chunk, column=input_column))
],
aggregate_funcs=[
(col, _apply_func_to_columns, dict(func=func.agg, prefix=col))
],
finalizer=finalizer
)
def _groupby_apply_funcs(df, *index, **kwargs):
"""
Group a dataframe and apply multiple aggregation functions.
Parameters
----------
df: pandas.DataFrame
The dataframe to work on.
index: list of groupers
If given, they are added to the keyword arguments as the ``by``
argument.
funcs: list of result-colum, function, keywordargument triples
The list of functions that are applied on the grouped data frame.
Has to be passed as a keyword argument.
kwargs:
All keyword arguments, but ``funcs``, are passed verbatim to the groupby
operation of the dataframe
Returns
-------
aggregated:
the aggregated dataframe.
"""
if len(index):
# 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
kwargs.update(by=list(index))
funcs = kwargs.pop('funcs')
grouped = _groupby_raise_unaligned(df, **kwargs)
result = collections.OrderedDict()
for result_column, func, func_kwargs in funcs:
r = func(grouped, **func_kwargs)
if isinstance(r, tuple):
for idx, s in enumerate(r):
result['{}-{}'.format(result_column, idx)] = s
else:
result[result_column] = r
return pd.DataFrame(result)
def _compute_sum_of_squares(grouped, column):
base = grouped[column] if column is not None else grouped
return base.apply(lambda x: (x ** 2).sum())
def _agg_finalize(df, aggregate_funcs, finalize_funcs, level):
# finish the final aggregation level
df = _groupby_apply_funcs(df, funcs=aggregate_funcs, level=level)
# and finalize the result
result = collections.OrderedDict()
for result_column, func, kwargs in finalize_funcs:
result[result_column] = func(df, **kwargs)
return pd.DataFrame(result)
def _apply_func_to_column(df_like, column, func):
if column is None:
return func(df_like)
return func(df_like[column])
def _apply_func_to_columns(df_like, prefix, func):
if isinstance(df_like, pd.DataFrame):
columns = df_like.columns
else:
# handle GroupBy objects
columns = df_like._selected_obj.columns
columns = sorted(col for col in columns if col.startswith(prefix))
columns = [df_like[col] for col in columns]
return func(*columns)
def _finalize_mean(df, sum_column, count_column):
return df[sum_column] / df[count_column]
def _finalize_var(df, count_column, sum_column, sum2_column, ddof=1):
n = df[count_column]
x = df[sum_column]
x2 = df[sum2_column]
result = x2 - x ** 2 / n
div = (n - ddof)
div[div < 0] = 0
result /= div
result[(n - ddof) == 0] = np.nan
return result
def _finalize_std(df, count_column, sum_column, sum2_column, ddof=1):
result = _finalize_var(df, count_column, sum_column, sum2_column, ddof)
return np.sqrt(result)
def _cum_agg_aligned(part, cum_last, index, columns, func, initial):
align = cum_last.reindex(part.set_index(index).index, fill_value=initial)
align.index = part.index
return func(part[columns], align)
def _cum_agg_filled(a, b, func, initial):
union = a.index.union(b.index)
return func(a.reindex(union, fill_value=initial),
b.reindex(union, fill_value=initial), fill_value=initial)
def _cumcount_aggregate(a, b, fill_value=None):
return a.add(b, fill_value=fill_value) + 1
class _GroupBy(object):
""" Superclass for DataFrameGroupBy and SeriesGroupBy
Parameters
----------
obj: DataFrame or Series
DataFrame or Series to be grouped
by: str, list or Series
The key for grouping
slice: str, list
The slice keys applied to GroupBy result
"""
def __init__(self, df, by=None, slice=None):
assert isinstance(df, (DataFrame, Series))
self.obj = df
# grouping key passed via groupby method
self.index = _normalize_index(df, by)
if isinstance(self.index, list):
do_index_partition_align = all(
item.divisions == df.divisions if isinstance(item, Series) else True
for item in self.index
)
elif isinstance(self.index, Series):
do_index_partition_align = df.divisions == self.index.divisions
else:
do_index_partition_align = True
if not do_index_partition_align:
raise NotImplementedError("The grouped object and index of the "
"groupby must have the same divisions.")
# slicing key applied to _GroupBy instance
self._slice = slice
if isinstance(self.index, list):
index_meta = [item._meta if isinstance(item, Series) else item for item in self.index]
elif isinstance(self.index, Series):
index_meta = self.index._meta
else:
index_meta = self.index
self._meta = self.obj._meta.groupby(index_meta)
@property
def _meta_nonempty(self):
"""
Return a pd.DataFrameGroupBy / pd.SeriesGroupBy which contains sample data.
"""
sample = self.obj._meta_nonempty
if isinstance(self.index, list):
index_meta = [item._meta_nonempty if isinstance(item, Series) else item for item in self.index]
elif isinstance(self.index, Series):
index_meta = self.index._meta_nonempty
else:
index_meta = self.index
grouped = sample.groupby(index_meta)
return _maybe_slice(grouped, self._slice)
def _aca_agg(self, token, func, aggfunc=None, split_every=None,
split_out=1):
if aggfunc is None:
aggfunc = func
meta = func(self._meta)
columns = meta.name if isinstance(meta, pd.Series) else meta.columns
token = self._token_prefix + token
levels = _determine_levels(self.index)
return aca([self.obj, self.index] if not isinstance(self.index, list) else [self.obj] + self.index,
chunk=_apply_chunk,
chunk_kwargs=dict(chunk=func, columns=columns),
aggregate=_groupby_aggregate,
meta=meta, token=token, split_every=split_every,
aggregate_kwargs=dict(aggfunc=aggfunc, levels=levels),
split_out=split_out, split_out_setup=split_out_on_index)
def _cum_agg(self, token, chunk, aggregate, initial):
""" Wrapper for cumulative groupby operation """
meta = chunk(self._meta)
columns = meta.name if isinstance(meta, pd.Series) else meta.columns
index = self.index if isinstance(self.index, list) else [self.index]
name = self._token_prefix + token
name_part = name + '-map'
name_last = name + '-take-last'
name_cum = name + '-cum-last'
# cumulate each partitions
cumpart_raw = map_partitions(_apply_chunk, self.obj, *index,
chunk=chunk,
columns=columns,
token=name_part,
meta=meta)
cumpart_raw_frame = (cumpart_raw.to_frame()
if isinstance(meta, pd.Series)
else cumpart_raw)
cumpart_ext = cumpart_raw_frame.assign(
**{i: self.obj[i]
if np.isscalar(i) and i in self.obj.columns
else self.obj.index
for i in index})
# Use pd.Grouper objects to specify that we are grouping by columns.
# Otherwise, pandas will throw an ambiguity warning if the
# DataFrame's index (self.obj.index) was included in the grouping
# specification (self.index). See pandas #14432
index_groupers = [pd.Grouper(key=ind) for ind in index]
cumlast = map_partitions(_apply_chunk, cumpart_ext, *index_groupers,
columns=0 if columns is None else columns,
chunk=M.last,
meta=meta,
token=name_last)
# aggregate cumulated partisions and its previous last element
dask = {}
dask[(name, 0)] = (cumpart_raw._name, 0)
for i in range(1, self.obj.npartitions):
# store each cumulative step to graph to reduce computation
if i == 1:
dask[(name_cum, i)] = (cumlast._name, i - 1)
else:
# aggregate with previous cumulation results
dask[(name_cum, i)] = (_cum_agg_filled,
(name_cum, i - 1),
(cumlast._name, i - 1),
aggregate, initial)
dask[(name, i)] = (_cum_agg_aligned,
(cumpart_ext._name, i), (name_cum, i),
index, 0 if columns is None else columns,
aggregate, initial)
return new_dd_object(merge(dask, cumpart_ext.dask, cumlast.dask),
name, chunk(self._meta), self.obj.divisions)
@derived_from(pd.core.groupby.GroupBy)
def cumsum(self, axis=0):
if axis:
return self.obj.cumsum(axis=axis)
else:
return self._cum_agg('cumsum',
chunk=M.cumsum,
aggregate=M.add,
initial=0)
@derived_from(pd.core.groupby.GroupBy)
def cumprod(self, axis=0):
if axis:
return self.obj.cumprod(axis=axis)
else:
return self._cum_agg('cumprod',
chunk=M.cumprod,
aggregate=M.mul,
initial=1)
@derived_from(pd.core.groupby.GroupBy)
def cumcount(self, axis=None):
return self._cum_agg('cumcount',
chunk=M.cumcount,
aggregate=_cumcount_aggregate,
initial=-1)
@derived_from(pd.core.groupby.GroupBy)
def sum(self, split_every=None, split_out=1):
return self._aca_agg(token='sum', func=M.sum, split_every=split_every,
split_out=split_out)
@derived_from(pd.core.groupby.GroupBy)
def min(self, split_every=None, split_out=1):
return self._aca_agg(token='min', func=M.min, split_every=split_every,
split_out=split_out)
@derived_from(pd.core.groupby.GroupBy)
def max(self, split_every=None, split_out=1):
return self._aca_agg(token='max', func=M.max, split_every=split_every,
split_out=split_out)
@derived_from(pd.core.groupby.GroupBy)
def count(self, split_every=None, split_out=1):
return self._aca_agg(token='count', func=M.count,
aggfunc=M.sum, split_every=split_every,
split_out=split_out)
@derived_from(pd.core.groupby.GroupBy)
def mean(self, split_every=None, split_out=1):
return (self.sum(split_every=split_every, split_out=split_out) /
self.count(split_every=split_every, split_out=split_out))
@derived_from(pd.core.groupby.GroupBy)
def size(self, split_every=None, split_out=1):
return self._aca_agg(token='size', func=M.size, aggfunc=M.sum,
split_every=split_every, split_out=split_out)
@derived_from(pd.core.groupby.GroupBy)
def var(self, ddof=1, split_every=None, split_out=1):
levels = _determine_levels(self.index)
result = aca([self.obj, self.index] if not isinstance(self.index, list) else [self.obj] + self.index,
chunk=_var_chunk,
aggregate=_var_agg, combine=_var_combine,
token=self._token_prefix + 'var',
aggregate_kwargs={'ddof': ddof, 'levels': levels},
combine_kwargs={'levels': levels},
split_every=split_every, split_out=split_out,
split_out_setup=split_out_on_index)
if isinstance(self.obj, Series):
result = result[result.columns[0]]
if self._slice:
result = result[self._slice]
return result
@derived_from(pd.core.groupby.GroupBy)
def std(self, ddof=1, split_every=None, split_out=1):
v = self.var(ddof, split_every=split_every, split_out=split_out)
result = map_partitions(np.sqrt, v, meta=v)
return result
@derived_from(pd.core.groupby.GroupBy)
def get_group(self, key):
token = self._token_prefix + 'get_group'
meta = self._meta.obj
if isinstance(meta, pd.DataFrame) and self._slice is not None:
meta = meta[self._slice]
columns = meta.columns if isinstance(meta, pd.DataFrame) else meta.name
return map_partitions(_groupby_get_group, self.obj, self.index, key,
columns, meta=meta, token=token)
def aggregate(self, arg, split_every, split_out=1):
if isinstance(self.obj, DataFrame):
if isinstance(self.index, tuple) or np.isscalar(self.index):
group_columns = {self.index}
elif isinstance(self.index, list):
group_columns = {i for i in self.index
if isinstance(i, tuple) or np.isscalar(i)}
else:
group_columns = set()
if self._slice:
# pandas doesn't exclude the grouping column in a SeriesGroupBy
# like df.groupby('a')['a'].agg(...)
non_group_columns = self._slice
if not isinstance(non_group_columns, list):
non_group_columns = [non_group_columns]
else:
# NOTE: this step relies on the index normalization to replace
# series with their name in an index.
non_group_columns = [col for col in self.obj.columns
if col not in group_columns]
spec = _normalize_spec(arg, non_group_columns)
elif isinstance(self.obj, Series):
if isinstance(arg, (list, tuple, dict)):
# implementation detail: if self.obj is a series, a pseudo column
# None is used to denote the series itself. This pseudo column is