-
-
Notifications
You must be signed in to change notification settings - Fork 1.7k
/
core.py
7196 lines (6101 loc) · 241 KB
/
core.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
import copy
import operator
import warnings
from collections.abc import Iterator, Sequence
from functools import partial, wraps
from numbers import Integral, Number
from operator import getitem
from pprint import pformat
import numpy as np
import pandas as pd
from pandas.api.types import (
is_bool_dtype,
is_datetime64_any_dtype,
is_numeric_dtype,
is_timedelta64_dtype,
)
from pandas.util import cache_readonly
from tlz import first, merge, partition_all, remove, unique
try:
from chest import Chest as Cache
except ImportError:
Cache = dict
from .. import array as da
from .. import core, threaded
from ..array.core import Array, normalize_arg
from ..base import DaskMethodsMixin, dont_optimize, is_dask_collection, tokenize
from ..blockwise import Blockwise, blockwise, subs
from ..context import globalmethod
from ..delayed import Delayed, delayed, unpack_collections
from ..highlevelgraph import HighLevelGraph
from ..optimization import SubgraphCallable
from ..utils import (
IndexCallable,
M,
OperatorMethodMixin,
_deprecated,
apply,
derived_from,
funcname,
has_keyword,
is_arraylike,
iter_chunks,
key_split,
memory_repr,
parse_bytes,
partial_by_order,
pseudorandom,
put_lines,
random_state_data,
typename,
)
from ..widgets import get_template
from . import methods
from .accessor import DatetimeAccessor, StringAccessor
from .categorical import CategoricalAccessor, categorize
from .dispatch import (
get_parallel_type,
group_split_dispatch,
hash_object_dispatch,
meta_nonempty,
)
from .optimize import optimize
from .utils import (
PANDAS_GT_110,
PANDAS_GT_120,
check_matching_columns,
clear_known_categories,
drop_by_shallow_copy,
has_known_categories,
index_summary,
insert_meta_param_description,
is_categorical_dtype,
is_dataframe_like,
is_index_like,
is_series_like,
make_meta,
raise_on_meta_error,
valid_divisions,
)
no_default = "__no_default__"
pd.set_option("compute.use_numexpr", False)
def _numeric_only(func):
"""Decorator for methods that accept a numeric_only kwarg"""
@wraps(func)
def wrapper(self, *args, **kwargs):
# numeric_only is None by default - in that case self = self.
if kwargs.get("numeric_only") is False:
raise NotImplementedError(
"'numeric_only=False' is not implemented in Dask."
)
elif kwargs.get("numeric_only") is True:
self = self._get_numeric_data()
return func(self, *args, **kwargs)
return wrapper
def _concat(args, ignore_index=False):
if not args:
return args
if isinstance(first(core.flatten(args)), np.ndarray):
return da.core.concatenate3(args)
if not has_parallel_type(args[0]):
try:
return pd.Series(args)
except Exception:
return args
# We filter out empty partitions here because pandas frequently has
# inconsistent dtypes in results between empty and non-empty frames.
# Ideally this would be handled locally for each operation, but in practice
# this seems easier. TODO: don't do this.
args2 = [i for i in args if len(i)]
return (
args[0]
if not args2
else methods.concat(args2, uniform=True, ignore_index=ignore_index)
)
def finalize(results):
return _concat(results)
class Scalar(DaskMethodsMixin, OperatorMethodMixin):
"""A Dask object to represent a pandas scalar"""
def __init__(self, dsk, name, meta, divisions=None):
# divisions is ignored, only present to be compatible with other
# objects.
if not isinstance(dsk, HighLevelGraph):
dsk = HighLevelGraph.from_collections(name, dsk, dependencies=[])
self.dask = dsk
self._name = name
self._parent_meta = pd.Series(dtype="float64")
meta = make_meta(meta, parent_meta=self._parent_meta)
if is_dataframe_like(meta) or is_series_like(meta) or is_index_like(meta):
raise TypeError(
"Expected meta to specify scalar, got "
"{0}".format(typename(type(meta)))
)
self._meta = meta
def __dask_graph__(self):
return self.dask
def __dask_keys__(self):
return [self.key]
def __dask_tokenize__(self):
return self._name
def __dask_layers__(self):
return (self._name,)
__dask_optimize__ = globalmethod(
optimize, key="dataframe_optimize", falsey=dont_optimize
)
__dask_scheduler__ = staticmethod(threaded.get)
def __dask_postcompute__(self):
return first, ()
def __dask_postpersist__(self):
return self._rebuild, ()
def _rebuild(self, dsk, *, rename=None):
name = self._name
if rename:
name = rename.get(name, name)
return Scalar(dsk, name, self._meta, self.divisions)
@property
def _meta_nonempty(self):
return self._meta
@property
def dtype(self):
return self._meta.dtype
def __dir__(self):
o = set(dir(type(self)))
o.update(self.__dict__)
if not hasattr(self._meta, "dtype"):
o.remove("dtype") # dtype only in `dir` if available
return list(o)
@property
def divisions(self):
"""Dummy divisions to be compat with Series and DataFrame"""
return [None, None]
def __repr__(self):
name = self._name if len(self._name) < 10 else self._name[:7] + "..."
if hasattr(self._meta, "dtype"):
extra = ", dtype=%s" % self._meta.dtype
else:
extra = ", type=%s" % type(self._meta).__name__
return "dd.Scalar<%s%s>" % (name, extra)
def __array__(self):
# array interface is required to support pandas instance + Scalar
# Otherwise, above op results in pd.Series of Scalar (object dtype)
return np.asarray(self.compute())
@property
def _args(self):
return (self.dask, self._name, self._meta)
def __getstate__(self):
return self._args
def __setstate__(self, state):
self.dask, self._name, self._meta = state
def __bool__(self):
raise TypeError(
"Trying to convert {} to a boolean value. Because Dask objects are "
"lazily evaluated, they cannot be converted to a boolean value or used "
"in boolean conditions like if statements. Try calling .compute() to "
"force computation prior to converting to a boolean value or using in "
"a conditional statement.".format(self)
)
@property
def key(self):
return (self._name, 0)
@classmethod
def _get_unary_operator(cls, op):
def f(self):
name = funcname(op) + "-" + tokenize(self)
dsk = {(name, 0): (op, (self._name, 0))}
meta = op(self._meta_nonempty)
graph = HighLevelGraph.from_collections(name, dsk, dependencies=[self])
return Scalar(graph, name, meta)
return f
@classmethod
def _get_binary_operator(cls, op, inv=False):
return lambda self, other: _scalar_binary(op, self, other, inv=inv)
def to_delayed(self, optimize_graph=True):
"""Convert into a ``dask.delayed`` object.
Parameters
----------
optimize_graph : bool, optional
If True [default], the graph is optimized before converting into
``dask.delayed`` objects.
"""
dsk = self.__dask_graph__()
if optimize_graph:
dsk = self.__dask_optimize__(dsk, self.__dask_keys__())
name = "delayed-" + self._name
dsk = HighLevelGraph.from_collections(name, dsk, dependencies=())
return Delayed(self.key, dsk)
def _scalar_binary(op, self, other, inv=False):
name = "{0}-{1}".format(funcname(op), tokenize(self, other))
dependencies = [self]
dsk = {}
return_type = get_parallel_type(other)
if isinstance(other, Scalar):
dependencies.append(other)
other_key = (other._name, 0)
elif is_dask_collection(other):
return NotImplemented
else:
other_key = other
dsk[(name, 0)] = (
(op, other_key, (self._name, 0)) if inv else (op, (self._name, 0), other_key)
)
other_meta = make_meta(other, parent_meta=self._parent_meta)
other_meta_nonempty = meta_nonempty(other_meta)
if inv:
meta = op(other_meta_nonempty, self._meta_nonempty)
else:
meta = op(self._meta_nonempty, other_meta_nonempty)
graph = HighLevelGraph.from_collections(name, dsk, dependencies=dependencies)
if return_type is not Scalar:
return return_type(graph, name, meta, [other.index.min(), other.index.max()])
else:
return Scalar(graph, name, meta)
class _Frame(DaskMethodsMixin, OperatorMethodMixin):
"""Superclass for DataFrame and Series
Parameters
----------
dsk: dict
The dask graph to compute this DataFrame
name: str
The key prefix that specifies which keys in the dask comprise this
particular DataFrame / Series
meta: pandas.DataFrame, pandas.Series, or pandas.Index
An empty pandas object with names, dtypes, and indices matching the
expected output.
divisions: tuple of index values
Values along which we partition our blocks on the index
"""
def __init__(self, dsk, name, meta, divisions):
if not isinstance(dsk, HighLevelGraph):
dsk = HighLevelGraph.from_collections(name, dsk, dependencies=[])
self.dask = dsk
self._name = name
meta = make_meta(meta)
if not self._is_partition_type(meta):
raise TypeError(
"Expected meta to specify type {0}, got type "
"{1}".format(type(self).__name__, typename(type(meta)))
)
self._meta = meta
self.divisions = tuple(divisions)
def __dask_graph__(self):
return self.dask
def __dask_keys__(self):
return [(self._name, i) for i in range(self.npartitions)]
def __dask_layers__(self):
return (self._name,)
def __dask_tokenize__(self):
return self._name
__dask_optimize__ = globalmethod(
optimize, key="dataframe_optimize", falsey=dont_optimize
)
__dask_scheduler__ = staticmethod(threaded.get)
def __dask_postcompute__(self):
return finalize, ()
def __dask_postpersist__(self):
return self._rebuild, ()
def _rebuild(self, dsk, *, rename=None):
name = self._name
if rename:
name = rename.get(name, name)
return type(self)(dsk, name, self._meta, self.divisions)
@property
def _constructor(self):
return new_dd_object
@property
def npartitions(self):
"""Return number of partitions"""
return len(self.divisions) - 1
@property
@derived_from(pd.DataFrame)
def attrs(self):
return self._meta.attrs
@attrs.setter
def attrs(self, value):
self._meta.attrs = dict(value)
@property
def size(self):
"""Size of the Series or DataFrame as a Delayed object.
Examples
--------
>>> series.size # doctest: +SKIP
dd.Scalar<size-ag..., dtype=int64>
"""
return self.reduction(
methods.size, np.sum, token="size", meta=int, split_every=False
)
@property
def _meta_nonempty(self):
"""A non-empty version of `_meta` with fake data."""
return meta_nonempty(self._meta)
@property
def _args(self):
return (self.dask, self._name, self._meta, self.divisions)
def __getstate__(self):
return self._args
def __setstate__(self, state):
self.dask, self._name, self._meta, self.divisions = state
def copy(self, deep=False):
"""Make a copy of the dataframe
This is strictly a shallow copy of the underlying computational graph.
It does not affect the underlying data
Parameters
----------
deep : boolean, default False
The deep value must be `False` and it is declared as a parameter just for
compatibility with third-party libraries like cuDF
"""
if deep is not False:
raise ValueError(
"The `deep` value must be False. This is strictly a shallow copy "
"of the underlying computational graph."
)
return new_dd_object(self.dask, self._name, self._meta, self.divisions)
def __array__(self, dtype=None, **kwargs):
self._computed = self.compute()
x = np.array(self._computed)
return x
def __array_wrap__(self, array, context=None):
raise NotImplementedError
def __array_ufunc__(self, numpy_ufunc, method, *inputs, **kwargs):
out = kwargs.get("out", ())
for x in inputs + out:
# ufuncs work with 0-dimensional NumPy ndarrays
# so we don't want to raise NotImplemented
if isinstance(x, np.ndarray) and x.shape == ():
continue
elif not isinstance(
x, (Number, Scalar, _Frame, Array, pd.DataFrame, pd.Series, pd.Index)
):
return NotImplemented
if method == "__call__":
if numpy_ufunc.signature is not None:
return NotImplemented
if numpy_ufunc.nout > 1:
# ufuncs with multiple output values
# are not yet supported for frames
return NotImplemented
else:
return elemwise(numpy_ufunc, *inputs, **kwargs)
else:
# ufunc methods are not yet supported for frames
return NotImplemented
@property
def _elemwise(self):
return elemwise
def _repr_data(self):
raise NotImplementedError
@property
def _repr_divisions(self):
name = "npartitions={0}".format(self.npartitions)
if self.known_divisions:
divisions = pd.Index(self.divisions, name=name)
else:
# avoid to be converted to NaN
divisions = pd.Index([""] * (self.npartitions + 1), name=name)
return divisions
def __repr__(self):
data = self._repr_data().to_string(max_rows=5, show_dimensions=False)
_str_fmt = """Dask {klass} Structure:
{data}
Dask Name: {name}, {task} tasks"""
if len(self.columns) == 0:
data = data.partition("\n")[-1].replace("Index", "Divisions")
_str_fmt = "Empty {}".format(_str_fmt)
return _str_fmt.format(
klass=self.__class__.__name__,
data=data,
name=key_split(self._name),
task=len(self.dask),
)
@property
def index(self):
"""Return dask Index instance"""
return self.map_partitions(
getattr,
"index",
token=self._name + "-index",
meta=self._meta.index,
enforce_metadata=False,
)
@index.setter
def index(self, value):
self.divisions = value.divisions
result = map_partitions(
methods.assign_index, self, value, enforce_metadata=False
)
self.dask = result.dask
self._name = result._name
self._meta = result._meta
def reset_index(self, drop=False):
"""Reset the index to the default index.
Note that unlike in ``pandas``, the reset ``dask.dataframe`` index will
not be monotonically increasing from 0. Instead, it will restart at 0
for each partition (e.g. ``index1 = [0, ..., 10], index2 = [0, ...]``).
This is due to the inability to statically know the full length of the
index.
For DataFrame with multi-level index, returns a new DataFrame with
labeling information in the columns under the index names, defaulting
to 'level_0', 'level_1', etc. if any are None. For a standard index,
the index name will be used (if set), otherwise a default 'index' or
'level_0' (if 'index' is already taken) will be used.
Parameters
----------
drop : boolean, default False
Do not try to insert index into dataframe columns.
"""
return self.map_partitions(
M.reset_index, drop=drop, enforce_metadata=False
).clear_divisions()
@property
def known_divisions(self):
"""Whether divisions are already known"""
return len(self.divisions) > 0 and self.divisions[0] is not None
def clear_divisions(self):
"""Forget division information"""
divisions = (None,) * (self.npartitions + 1)
return type(self)(self.dask, self._name, self._meta, divisions)
def get_partition(self, n):
"""Get a dask DataFrame/Series representing the `nth` partition."""
if 0 <= n < self.npartitions:
name = "get-partition-%s-%s" % (str(n), self._name)
divisions = self.divisions[n : n + 2]
layer = {(name, 0): (self._name, n)}
graph = HighLevelGraph.from_collections(name, layer, dependencies=[self])
return new_dd_object(graph, name, self._meta, divisions)
else:
msg = "n must be 0 <= n < {0}".format(self.npartitions)
raise ValueError(msg)
@derived_from(pd.DataFrame)
def drop_duplicates(
self, subset=None, split_every=None, split_out=1, ignore_index=False, **kwargs
):
if subset is not None:
# Let pandas error on bad inputs
self._meta_nonempty.drop_duplicates(subset=subset, **kwargs)
kwargs["subset"] = subset
split_out_setup = split_out_on_cols
split_out_setup_kwargs = {"cols": subset}
else:
self._meta_nonempty.drop_duplicates(**kwargs)
split_out_setup = split_out_setup_kwargs = None
if kwargs.get("keep", True) is False:
raise NotImplementedError("drop_duplicates with keep=False")
chunk = M.drop_duplicates
return aca(
self,
chunk=chunk,
aggregate=chunk,
meta=self._meta,
token="drop-duplicates",
split_every=split_every,
split_out=split_out,
split_out_setup=split_out_setup,
split_out_setup_kwargs=split_out_setup_kwargs,
ignore_index=ignore_index,
**kwargs,
)
def __len__(self):
return self.reduction(
len, np.sum, token="len", meta=int, split_every=False
).compute()
def __bool__(self):
raise ValueError(
"The truth value of a {0} is ambiguous. "
"Use a.any() or a.all().".format(self.__class__.__name__)
)
__nonzero__ = __bool__ # python 2
def _scalarfunc(self, cast_type):
def wrapper():
raise TypeError("cannot convert the series to {0}".format(str(cast_type)))
return wrapper
def __float__(self):
return self._scalarfunc(float)
def __int__(self):
return self._scalarfunc(int)
__long__ = __int__ # python 2
def __complex__(self):
return self._scalarfunc(complex)
@insert_meta_param_description(pad=12)
def map_partitions(self, func, *args, **kwargs):
"""Apply Python function on each DataFrame partition.
Note that the index and divisions are assumed to remain unchanged.
Parameters
----------
func : function
Function applied to each partition.
args, kwargs :
Arguments and keywords to pass to the function. The partition will
be the first argument, and these will be passed *after*. Arguments
and keywords may contain ``Scalar``, ``Delayed``, ``partition_info``
or regular python objects. DataFrame-like args (both dask and
pandas) will be repartitioned to align (if necessary) before
applying the function.
$META
Examples
--------
Given a DataFrame, Series, or Index, such as:
>>> import pandas as pd
>>> import dask.dataframe as dd
>>> df = pd.DataFrame({'x': [1, 2, 3, 4, 5],
... 'y': [1., 2., 3., 4., 5.]})
>>> ddf = dd.from_pandas(df, npartitions=2)
One can use ``map_partitions`` to apply a function on each partition.
Extra arguments and keywords can optionally be provided, and will be
passed to the function after the partition.
Here we apply a function with arguments and keywords to a DataFrame,
resulting in a Series:
>>> def myadd(df, a, b=1):
... return df.x + df.y + a + b
>>> res = ddf.map_partitions(myadd, 1, b=2)
>>> res.dtype
dtype('float64')
By default, dask tries to infer the output metadata by running your
provided function on some fake data. This works well in many cases, but
can sometimes be expensive, or even fail. To avoid this, you can
manually specify the output metadata with the ``meta`` keyword. This
can be specified in many forms, for more information see
``dask.dataframe.utils.make_meta``.
Here we specify the output is a Series with no name, and dtype
``float64``:
>>> res = ddf.map_partitions(myadd, 1, b=2, meta=(None, 'f8'))
Here we map a function that takes in a DataFrame, and returns a
DataFrame with a new column:
>>> res = ddf.map_partitions(lambda df: df.assign(z=df.x * df.y))
>>> res.dtypes
x int64
y float64
z float64
dtype: object
As before, the output metadata can also be specified manually. This
time we pass in a ``dict``, as the output is a DataFrame:
>>> res = ddf.map_partitions(lambda df: df.assign(z=df.x * df.y),
... meta={'x': 'i8', 'y': 'f8', 'z': 'f8'})
In the case where the metadata doesn't change, you can also pass in
the object itself directly:
>>> res = ddf.map_partitions(lambda df: df.head(), meta=ddf)
Also note that the index and divisions are assumed to remain unchanged.
If the function you're mapping changes the index/divisions, you'll need
to clear them afterwards:
>>> ddf.map_partitions(func).clear_divisions() # doctest: +SKIP
Your map function gets information about where it is in the dataframe by
accepting a special ``partition_info`` keyword argument.
>>> def func(partition, partition_info=None):
... pass
This will receive the following information:
>>> partition_info # doctest: +SKIP
{'number': 1, 'division': 3}
For each argument and keyword arguments that are dask dataframes you will
receive the number (n) which represents the nth partition of the dataframe
and the division (the first index value in the partition). If divisions
are not known (for instance if the index is not sorted) then you will get
None as the division.
"""
return map_partitions(func, self, *args, **kwargs)
@insert_meta_param_description(pad=12)
def map_overlap(self, func, before, after, *args, **kwargs):
"""Apply a function to each partition, sharing rows with adjacent partitions.
This can be useful for implementing windowing functions such as
``df.rolling(...).mean()`` or ``df.diff()``.
Parameters
----------
func : function
Function applied to each partition.
before : int
The number of rows to prepend to partition ``i`` from the end of
partition ``i - 1``.
after : int
The number of rows to append to partition ``i`` from the beginning
of partition ``i + 1``.
args, kwargs :
Arguments and keywords to pass to the function. The partition will
be the first argument, and these will be passed *after*.
$META
Notes
-----
Given positive integers ``before`` and ``after``, and a function
``func``, ``map_overlap`` does the following:
1. Prepend ``before`` rows to each partition ``i`` from the end of
partition ``i - 1``. The first partition has no rows prepended.
2. Append ``after`` rows to each partition ``i`` from the beginning of
partition ``i + 1``. The last partition has no rows appended.
3. Apply ``func`` to each partition, passing in any extra ``args`` and
``kwargs`` if provided.
4. Trim ``before`` rows from the beginning of all but the first
partition.
5. Trim ``after`` rows from the end of all but the last partition.
Note that the index and divisions are assumed to remain unchanged.
Examples
--------
Given a DataFrame, Series, or Index, such as:
>>> import pandas as pd
>>> import dask.dataframe as dd
>>> df = pd.DataFrame({'x': [1, 2, 4, 7, 11],
... 'y': [1., 2., 3., 4., 5.]})
>>> ddf = dd.from_pandas(df, npartitions=2)
A rolling sum with a trailing moving window of size 2 can be computed by
overlapping 2 rows before each partition, and then mapping calls to
``df.rolling(2).sum()``:
>>> ddf.compute()
x y
0 1 1.0
1 2 2.0
2 4 3.0
3 7 4.0
4 11 5.0
>>> ddf.map_overlap(lambda df: df.rolling(2).sum(), 2, 0).compute()
x y
0 NaN NaN
1 3.0 3.0
2 6.0 5.0
3 11.0 7.0
4 18.0 9.0
The pandas ``diff`` method computes a discrete difference shifted by a
number of periods (can be positive or negative). This can be
implemented by mapping calls to ``df.diff`` to each partition after
prepending/appending that many rows, depending on sign:
>>> def diff(df, periods=1):
... before, after = (periods, 0) if periods > 0 else (0, -periods)
... return df.map_overlap(lambda df, periods=1: df.diff(periods),
... periods, 0, periods=periods)
>>> diff(ddf, 1).compute()
x y
0 NaN NaN
1 1.0 1.0
2 2.0 1.0
3 3.0 1.0
4 4.0 1.0
If you have a ``DatetimeIndex``, you can use a ``pd.Timedelta`` for time-
based windows.
>>> ts = pd.Series(range(10), index=pd.date_range('2017', periods=10))
>>> dts = dd.from_pandas(ts, npartitions=2)
>>> dts.map_overlap(lambda df: df.rolling('2D').sum(),
... pd.Timedelta('2D'), 0).compute()
2017-01-01 0.0
2017-01-02 1.0
2017-01-03 3.0
2017-01-04 5.0
2017-01-05 7.0
2017-01-06 9.0
2017-01-07 11.0
2017-01-08 13.0
2017-01-09 15.0
2017-01-10 17.0
Freq: D, dtype: float64
"""
from .rolling import map_overlap
return map_overlap(func, self, before, after, *args, **kwargs)
def memory_usage_per_partition(self, index=True, deep=False):
"""Return the memory usage of each partition
Parameters
----------
index : bool, default True
Specifies whether to include the memory usage of the index in
returned Series.
deep : bool, default False
If True, introspect the data deeply by interrogating
``object`` dtypes for system-level memory consumption, and include
it in the returned values.
Returns
-------
Series
A Series whose index is the partition number and whose values
are the memory usage of each partition in bytes.
"""
return self.map_partitions(
total_mem_usage, index=index, deep=deep
).clear_divisions()
@insert_meta_param_description(pad=12)
def reduction(
self,
chunk,
aggregate=None,
combine=None,
meta=no_default,
token=None,
split_every=None,
chunk_kwargs=None,
aggregate_kwargs=None,
combine_kwargs=None,
**kwargs,
):
"""Generic row-wise reductions.
Parameters
----------
chunk : callable
Function to operate on each partition. Should return a
``pandas.DataFrame``, ``pandas.Series``, or a scalar.
aggregate : callable, optional
Function to operate on the concatenated result of ``chunk``. If not
specified, defaults to ``chunk``. Used to do the final aggregation
in a tree reduction.
The input to ``aggregate`` depends on the output of ``chunk``.
If the output of ``chunk`` is a:
- scalar: Input is a Series, with one row per partition.
- Series: Input is a DataFrame, with one row per partition. Columns
are the rows in the output series.
- DataFrame: Input is a DataFrame, with one row per partition.
Columns are the columns in the output dataframes.
Should return a ``pandas.DataFrame``, ``pandas.Series``, or a
scalar.
combine : callable, optional
Function to operate on intermediate concatenated results of
``chunk`` in a tree-reduction. If not provided, defaults to
``aggregate``. The input/output requirements should match that of
``aggregate`` described above.
$META
token : str, optional
The name to use for the output keys.
split_every : int, optional
Group partitions into groups of this size while performing a
tree-reduction. If set to False, no tree-reduction will be used,
and all intermediates will be concatenated and passed to
``aggregate``. Default is 8.
chunk_kwargs : dict, optional
Keyword arguments to pass on to ``chunk`` only.
aggregate_kwargs : dict, optional
Keyword arguments to pass on to ``aggregate`` only.
combine_kwargs : dict, optional
Keyword arguments to pass on to ``combine`` only.
kwargs :
All remaining keywords will be passed to ``chunk``, ``combine``,
and ``aggregate``.
Examples
--------
>>> import pandas as pd
>>> import dask.dataframe as dd
>>> df = pd.DataFrame({'x': range(50), 'y': range(50, 100)})
>>> ddf = dd.from_pandas(df, npartitions=4)
Count the number of rows in a DataFrame. To do this, count the number
of rows in each partition, then sum the results:
>>> res = ddf.reduction(lambda x: x.count(),
... aggregate=lambda x: x.sum())
>>> res.compute()
x 50
y 50
dtype: int64
Count the number of rows in a Series with elements greater than or
equal to a value (provided via a keyword).
>>> def count_greater(x, value=0):
... return (x >= value).sum()
>>> res = ddf.x.reduction(count_greater, aggregate=lambda x: x.sum(),
... chunk_kwargs={'value': 25})
>>> res.compute()
25
Aggregate both the sum and count of a Series at the same time:
>>> def sum_and_count(x):
... return pd.Series({'count': x.count(), 'sum': x.sum()},
... index=['count', 'sum'])
>>> res = ddf.x.reduction(sum_and_count, aggregate=lambda x: x.sum())
>>> res.compute()
count 50
sum 1225
dtype: int64
Doing the same, but for a DataFrame. Here ``chunk`` returns a
DataFrame, meaning the input to ``aggregate`` is a DataFrame with an
index with non-unique entries for both 'x' and 'y'. We groupby the
index, and sum each group to get the final result.
>>> def sum_and_count(x):
... return pd.DataFrame({'count': x.count(), 'sum': x.sum()},
... columns=['count', 'sum'])
>>> res = ddf.reduction(sum_and_count,
... aggregate=lambda x: x.groupby(level=0).sum())
>>> res.compute()
count sum
x 50 1225
y 50 3725
"""
if aggregate is None:
aggregate = chunk
if combine is None:
if combine_kwargs:
raise ValueError("`combine_kwargs` provided with no `combine`")
combine = aggregate
combine_kwargs = aggregate_kwargs
chunk_kwargs = chunk_kwargs.copy() if chunk_kwargs else {}
chunk_kwargs["aca_chunk"] = chunk
combine_kwargs = combine_kwargs.copy() if combine_kwargs else {}
combine_kwargs["aca_combine"] = combine
aggregate_kwargs = aggregate_kwargs.copy() if aggregate_kwargs else {}
aggregate_kwargs["aca_aggregate"] = aggregate
return aca(
self,
chunk=_reduction_chunk,
aggregate=_reduction_aggregate,
combine=_reduction_combine,
meta=meta,
token=token,
split_every=split_every,
chunk_kwargs=chunk_kwargs,
aggregate_kwargs=aggregate_kwargs,
combine_kwargs=combine_kwargs,
**kwargs,
)