-
Notifications
You must be signed in to change notification settings - Fork 2
/
core.py
3076 lines (2478 loc) · 103 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
from __future__ import absolute_import, division, print_function
import bisect
from collections import Iterator
from datetime import datetime
from distutils.version import LooseVersion
import math
import operator
from operator import getitem, setitem
from pprint import pformat
import uuid
import warnings
from toolz import merge, partial, first, partition, unique, merge_sorted, take
import pandas as pd
from pandas.util.decorators import cache_readonly
import numpy as np
try:
from chest import Chest as Cache
except ImportError:
Cache = dict
from .. import array as da
from .. import core
from ..array.core import partial_by_order
from .. import threaded
from ..compatibility import apply, operator_div, bind_method
from ..utils import (repr_long_list, IndexCallable,
pseudorandom, derived_from, different_seeds, funcname)
from ..base import Base, compute, tokenize, normalize_token
from ..async import get_sync
from .utils import nonempty_sample_df
no_default = '__no_default__'
return_scalar = '__return_scalar__'
pd.computation.expressions.set_use_numexpr(False)
def _concat(args, **kwargs):
""" Generic concat operation """
if not args:
return args
if isinstance(first(core.flatten(args)), np.ndarray):
return da.core.concatenate3(args)
if isinstance(args[0], (pd.DataFrame, pd.Series)):
args2 = [arg for arg in args if len(arg)]
if not args2:
return args[0]
return pd.concat(args2)
if isinstance(args[0], (pd.Index)):
args = [arg for arg in args if len(arg)]
return args[0].append(args[1:])
return args
def optimize(dsk, keys, **kwargs):
from .optimize import optimize
return optimize(dsk, keys, **kwargs)
def finalize(results):
return _concat(results)
class Scalar(Base):
""" A Dask-thing to represent a scalar
TODO: Clean up this abstraction
"""
_optimize = staticmethod(optimize)
_default_get = staticmethod(threaded.get)
_finalize = staticmethod(first)
def __init__(self, dsk, _name, name=None, divisions=None):
self.dask = dsk
self._name = _name
self.divisions = [None, None]
# name and divisions are ignored.
# There are dummies to be compat with Series and DataFrame
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)
@property
def key(self):
return (self._name, 0)
def _keys(self):
return [self.key]
@classmethod
def _get_unary_operator(cls, op):
def f(self):
name = funcname(op) + '-' + tokenize(self)
dsk = {(name, 0): (op, (self._name, 0))}
return Scalar(merge(dsk, self.dask), name)
return f
@classmethod
def _get_binary_operator(cls, op, inv=False):
return lambda self, other: _scalar_binary(op, self, other, inv=inv)
def _scalar_binary(op, a, b, inv=False):
name = '{0}-{1}'.format(funcname(op), tokenize(a, b))
dsk = a.dask
if not isinstance(b, Base):
pass
elif isinstance(b, Scalar):
dsk = merge(dsk, b.dask)
b = (b._name, 0)
else:
return NotImplemented
if inv:
dsk.update({(name, 0): (op, b, (a._name, 0))})
else:
dsk.update({(name, 0): (op, (a._name, 0), b)})
if isinstance(b, (pd.Series, pd.DataFrame)):
return _Frame(dsk, name, b, [b.index.min(), b.index.max()])
else:
return Scalar(dsk, name)
class _Frame(Base):
""" 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
metadata: scalar, None, list, pandas.Series or pandas.DataFrame
metadata to specify data structure.
- If scalar or None is given, the result is Series.
- If list is given, the result is DataFrame.
- If pandas data is given, the result is the class corresponding to
pandas data.
divisions: tuple of index values
Values along which we partition our blocks on the index
"""
_optimize = staticmethod(optimize)
_default_get = staticmethod(threaded.get)
_finalize = staticmethod(finalize)
def __new__(cls, dsk, _name, metadata, divisions):
if (np.isscalar(metadata) or metadata is None or
isinstance(metadata, (Series, pd.Series))):
return Series(dsk, _name, metadata, divisions)
else:
return DataFrame(dsk, _name, metadata, divisions)
# constructor properties
# http://pandas.pydata.org/pandas-docs/stable/internals.html#override-constructor-properties
@property
def _constructor_sliced(self):
"""Constructor used when a result has one lower dimension(s) as the original"""
raise NotImplementedError
@property
def _constructor(self):
"""Constructor used when a result has the same dimension(s) as the original"""
raise NotImplementedError
@property
def npartitions(self):
"""Return number of partitions"""
return len(self.divisions) - 1
@classmethod
def _build_pd(cls, metadata):
""" build pandas instance from passed metadata """
if isinstance(metadata, cls):
# copy metadata
_pd = metadata._pd
known_dtype = metadata._known_dtype
elif isinstance(metadata, cls._partition_type):
if isinstance(metadata, pd.Index):
_pd = metadata[0:0]
else:
_pd = metadata.iloc[0:0]
known_dtype = True
else:
if np.isscalar(metadata) or metadata is None:
_pd = cls._partition_type([], name=metadata)
else:
_pd = cls._partition_type(columns=metadata)
known_dtype = False
return _pd, known_dtype
@property
def _args(self):
return NotImplementedError
def __getnewargs__(self):
""" To load pickle """
return self._args
def _keys(self):
return [(self._name, i) for i in range(self.npartitions)]
def __repr__(self):
name = self._name if len(self._name) < 10 else self._name[:7] + '...'
if self.known_divisions:
div_text = ', divisions=%s' % repr_long_list(self.divisions)
else:
div_text = ''
return ("dd.%s<%s, npartitions=%s%s>" %
(self.__class__.__name__, name, self.npartitions, div_text))
@property
def index(self):
"""Return dask Index instance"""
name = self._name + '-index'
dsk = dict(((name, i), (getattr, key, 'index'))
for i, key in enumerate(self._keys()))
return Index(merge(dsk, self.dask), name,
self._pd.index.name, self.divisions)
@property
def known_divisions(self):
"""Whether divisions are already known"""
return len(self.divisions) > 0 and self.divisions[0] is not None
def get_division(self, n):
""" Get nth division of the data """
if 0 <= n < self.npartitions:
name = 'get-division-%s-%s' % (str(n), self._name)
dsk = {(name, 0): (self._name, n)}
divisions = self.divisions[n:n+2]
return self._constructor(merge(self.dask, dsk), name,
self._pd, divisions)
else:
msg = "n must be 0 <= n < {0}".format(self.npartitions)
raise ValueError(msg)
def cache(self, cache=Cache):
""" Evaluate Dataframe and store in local cache
Uses chest by default to store data on disk
"""
if callable(cache):
cache = cache()
# Evaluate and store in cache
name = 'cache' + uuid.uuid1().hex
dsk = dict(((name, i), (setitem, cache, (tuple, list(key)), key))
for i, key in enumerate(self._keys()))
self._get(merge(dsk, self.dask), list(dsk.keys()))
# Create new dataFrame pointing to that cache
name = 'from-cache-' + self._name
dsk2 = dict(((name, i), (getitem, cache, (tuple, list(key))))
for i, key in enumerate(self._keys()))
return self._constructor(dsk2, name, self._pd, self.divisions)
@derived_from(pd.DataFrame)
def drop_duplicates(self, **kwargs):
assert all(k in ('keep', 'subset', 'take_last') for k in kwargs)
chunk = lambda s: s.drop_duplicates(**kwargs)
return aca(self, chunk=chunk, aggregate=chunk, columns=self._pd,
token='drop-duplicates')
def __len__(self):
return reduction(self, len, np.sum, token='len').compute()
def map_partitions(self, func, columns=no_default, *args, **kwargs):
""" Apply Python function on each DataFrame block
When using ``map_partitions`` you should provide either the column
names (if the result is a DataFrame) or the name of the Series (if the
result is a Series). The output type will be determined by the type of
``columns``.
Parameters
----------
func : function
Function applied to each blocks
columns : tuple or scalar
Column names or name of the output. Defaults to names of data itself.
When tuple is passed, DataFrame is returned. When scalar is passed,
Series is returned.
Examples
--------
When str is passed as columns, the result will be Series.
>>> df.map_partitions(lambda df: df.x + 1, columns='x') # doctest: +SKIP
When tuple is passed as columns, the result will be Series.
>>> df.map_partitions(lambda df: df.head(), columns=df.columns) # doctest: +SKIP
"""
return map_partitions(func, columns, self, *args, **kwargs)
def random_split(self, p, random_state=None):
""" Pseudorandomly split dataframe into different pieces row-wise
Parameters
----------
frac : float, optional
Fraction of axis items to return.
random_state: int or np.random.RandomState
If int create a new RandomState with this as the seed
Otherwise draw from the passed RandomState
Examples
--------
50/50 split
>>> a, b = df.random_split([0.5, 0.5]) # doctest: +SKIP
80/10/10 split, consistent random_state
>>> a, b, c = df.random_split([0.8, 0.1, 0.1], random_state=123) # doctest: +SKIP
See Also
--------
dask.DataFrame.sample
"""
seeds = different_seeds(self.npartitions, random_state)
dsk_full = dict(((self._name + '-split-full', i),
(pd_split, (self._name, i), p, seed))
for i, seed in enumerate(seeds))
dsks = [dict(((self._name + '-split-%d' % i, j),
(getitem, (self._name + '-split-full', j), i))
for j in range(self.npartitions))
for i in range(len(p))]
return [type(self)(merge(self.dask, dsk_full, dsk),
self._name + '-split-%d' % i,
self._pd, self.divisions)
for i, dsk in enumerate(dsks)]
def head(self, n=5, compute=True):
""" First n rows of the dataset
Caveat, this only checks the first n rows of the first partition.
"""
name = 'head-%d-%s' % (n, self._name)
dsk = {(name, 0): (lambda x, n: x.head(n=n), (self._name, 0), n)}
result = self._constructor(merge(self.dask, dsk), name,
self._pd, self.divisions[:2])
if compute:
result = result.compute()
return result
def tail(self, n=5, compute=True):
""" Last n rows of the dataset
Caveat, the only checks the last n rows of the last partition.
"""
name = 'tail-%d-%s' % (n, self._name)
dsk = {(name, 0): (lambda x, n: x.tail(n=n),
(self._name, self.npartitions - 1), n)}
result = self._constructor(merge(self.dask, dsk), name,
self._pd, self.divisions[-2:])
if compute:
result = result.compute()
return result
def _loc(self, ind):
""" Helper function for the .loc accessor """
if isinstance(ind, Series):
return self._loc_series(ind)
if self.known_divisions:
if isinstance(ind, slice):
return self._loc_slice(ind)
else:
return self._loc_element(ind)
else:
return map_partitions(try_loc, self, self, ind)
def _loc_series(self, ind):
if not self.divisions == ind.divisions:
raise ValueError("Partitions of dataframe and index not the same")
return map_partitions(lambda df, ind: df.loc[ind],
self._pd, self, ind, token='loc-series')
def _loc_element(self, ind):
name = 'loc-%s' % tokenize(ind, self)
part = _partition_of_index_value(self.divisions, ind)
if ind < self.divisions[0] or ind > self.divisions[-1]:
raise KeyError('the label [%s] is not in the index' % str(ind))
dsk = {(name, 0): (lambda df: df.loc[ind:ind], (self._name, part))}
return self._constructor(merge(self.dask, dsk), name, self, [ind, ind])
def _loc_slice(self, ind):
name = 'loc-%s' % tokenize(ind, self)
assert ind.step in (None, 1)
if ind.start:
start = _partition_of_index_value(self.divisions, ind.start)
else:
start = 0
if ind.stop is not None:
stop = _partition_of_index_value(self.divisions, ind.stop)
else:
stop = self.npartitions - 1
istart = _coerce_loc_index(self.divisions, ind.start)
istop = _coerce_loc_index(self.divisions, ind.stop)
if stop == start:
dsk = {(name, 0): (_loc, (self._name, start), ind.start, ind.stop)}
divisions = [istart, istop]
else:
dsk = merge(
{(name, 0): (_loc, (self._name, start), ind.start, None)},
dict(((name, i), (self._name, start + i))
for i in range(1, stop - start)),
{(name, stop - start): (_loc, (self._name, stop), None, ind.stop)})
divisions = ((max(istart, self.divisions[start])
if ind.start is not None
else self.divisions[0],) +
self.divisions[start+1:stop+1] +
(min(istop, self.divisions[stop+1])
if ind.stop is not None
else self.divisions[-1],))
assert len(divisions) == len(dsk) + 1
return self._constructor(merge(self.dask, dsk), name,
self._pd, divisions)
@property
def loc(self):
""" Purely label-location based indexer for selection by label.
>>> df.loc["b"] # doctest: +SKIP
>>> df.loc["b":"d"] # doctest: +SKIP"""
return IndexCallable(self._loc)
# NOTE: `iloc` is not implemented because of performance concerns.
# see https://github.com/dask/dask/pull/507
def repartition(self, divisions=None, npartitions=None, force=False):
""" Repartition dataframe along new divisions
Parameters
----------
divisions : list
List of partitions to be used
npartitions : int
Number of partitions of output, must be less than npartitions of
input
force : bool, default False
Allows the expansion of the existing divisions.
If False then the new divisions lower and upper bounds must be
the same as the old divisions.
Examples
--------
>>> df = df.repartition(npartitions=10) # doctest: +SKIP
>>> df = df.repartition(divisions=[0, 5, 10, 20]) # doctest: +SKIP
"""
if npartitions is not None:
if npartitions > self.npartitions:
raise ValueError("Can only repartition to fewer partitions")
return repartition_npartitions(self, npartitions)
elif divisions is not None:
return repartition(self, divisions, force=force)
else:
raise ValueError(
"Provide either divisions= or npartitions= to repartition")
def __getstate__(self):
return self.__dict__
def __setstate__(self, dict):
self.__dict__ = dict
@derived_from(pd.Series)
def fillna(self, value):
return self.map_partitions(self._partition_type.fillna, value=value)
def sample(self, frac, replace=False, random_state=None):
""" Random sample of items
Parameters
----------
frac : float, optional
Fraction of axis items to return.
replace: boolean, optional
Sample with or without replacement. Default = False.
random_state: int or ``np.random.RandomState``
If int we create a new RandomState with this as the seed
Otherwise we draw from the passed RandomState
See Also
--------
dask.DataFrame.random_split, pd.DataFrame.sample
"""
if random_state is None:
random_state = np.random.randint(np.iinfo(np.int32).max)
name = 'sample-' + tokenize(self, frac, replace, random_state)
func = getattr(self._partition_type, 'sample')
seeds = different_seeds(self.npartitions, random_state)
dsk = dict(((name, i),
(apply, func, (tuple, [(self._name, i)]),
{'frac': frac, 'random_state': seed,
'replace': replace}))
for i, seed in zip(range(self.npartitions), seeds))
return self._constructor(merge(self.dask, dsk), name,
self._pd, self.divisions)
@derived_from(pd.DataFrame)
def to_hdf(self, path_or_buf, key, mode='a', append=False, complevel=0,
complib=None, fletcher32=False, get=get_sync, **kwargs):
from .io import to_hdf
return to_hdf(self, path_or_buf, key, mode, append, complevel, complib,
fletcher32, get=get, **kwargs)
@derived_from(pd.DataFrame)
def to_csv(self, filename, get=get_sync, **kwargs):
from .io import to_csv
return to_csv(self, filename, get=get, **kwargs)
def to_imperative(self):
warnings.warn("Deprecation warning: moved to to_delayed")
return self.to_delayed()
def to_delayed(self):
""" Convert dataframe into dask Values
Returns a list of values, one value per partition.
"""
from ..delayed import Delayed
return [Delayed(k, [self.dask]) for k in self._keys()]
@classmethod
def _get_unary_operator(cls, op):
return lambda self: elemwise(op, self)
@classmethod
def _get_binary_operator(cls, op, inv=False):
if inv:
return lambda self, other: elemwise(op, other, self)
else:
return lambda self, other: elemwise(op, self, other)
def _aca_agg(self, token, func, aggfunc=None, **kwargs):
""" Wrapper for aggregations """
raise NotImplementedError
def rolling(self, window, min_periods=None, win_type=None, axis=0):
"""Provides rolling transformations.
Parameters
----------
window: int
Size of the moving window. This window must be smaller
than the size of the previous partition.
min_periods: int, default None
Minimum number of observations in window required to have
a value (otherwise result is NA).
win_type: string, default None
Provide a window type. (Identical to pandas.)
axis: int, default 0
Provide the axis to apply the function. (Identical to pandas.)
The center argument and deprecated freq argument are not supported.
"""
from dask.dataframe.rolling import Rolling
if not isinstance(window, int):
raise ValueError('window must be an integer')
if window < 0:
raise ValueError('window must be >= 0')
if min_periods is not None:
if not isinstance(min_periods, int):
raise ValueError('min_periods must be an integer')
if min_periods < 0:
raise ValueError('min_periods must be >= 0')
return Rolling(self, window=window, min_periods=min_periods,
axis=axis, win_type=win_type)
@derived_from(pd.DataFrame)
def sum(self, axis=None, skipna=True):
axis = self._validate_axis(axis)
if axis == 1:
return map_partitions(_sum, None, self,
token=self._token_prefix + 'sum',
axis=axis, skipna=skipna)
else:
return self._aca_agg(token='sum', func=_sum,
skipna=skipna, axis=axis)
@derived_from(pd.DataFrame)
def max(self, axis=None, skipna=True):
axis = self._validate_axis(axis)
if axis == 1:
return map_partitions(_max, None, self,
token=self._token_prefix + 'max',
skipna=skipna, axis=axis)
else:
return self._aca_agg(token='max', func=_max,
skipna=skipna, axis=axis)
@derived_from(pd.DataFrame)
def min(self, axis=None, skipna=True):
axis = self._validate_axis(axis)
if axis == 1:
return map_partitions(_min, None, self,
token=self._token_prefix + 'min',
skipna=skipna, axis=axis)
else:
return self._aca_agg(token='min', func=_min,
skipna=skipna, axis=axis)
@derived_from(pd.DataFrame)
def count(self, axis=None):
axis = self._validate_axis(axis)
if axis == 1:
return map_partitions(_count, None, self,
token=self._token_prefix + 'count',
axis=axis)
else:
return self._aca_agg(token='count', func=_count,
aggfunc=lambda x: x.sum())
@derived_from(pd.DataFrame)
def mean(self, axis=None, skipna=True):
axis = self._validate_axis(axis)
if axis == 1:
return map_partitions(_mean, None, self,
token=self._token_prefix + 'mean',
axis=axis, skipna=skipna)
else:
num = self._get_numeric_data()
s = num.sum(skipna=skipna)
n = num.count()
def f(s, n):
try:
return s / n
except ZeroDivisionError:
return np.nan
name = self._token_prefix + 'mean-%s' % tokenize(self, axis, skipna)
return map_partitions(f, None, s, n, token=name)
@derived_from(pd.DataFrame)
def var(self, axis=None, skipna=True, ddof=1):
axis = self._validate_axis(axis)
if axis == 1:
return map_partitions(_var, None, self,
token=self._token_prefix + 'var',
axis=axis, skipna=skipna, ddof=ddof)
else:
num = self._get_numeric_data()
x = 1.0 * num.sum(skipna=skipna)
x2 = 1.0 * (num ** 2).sum(skipna=skipna)
n = num.count()
def f(x2, x, n):
try:
result = (x2 / n) - (x / n)**2
if ddof:
result = result * n / (n - ddof)
return result
except ZeroDivisionError:
return np.nan
name = self._token_prefix + 'var-%s' % tokenize(self, axis, skipna, ddof)
return map_partitions(f, None, x2, x, n, token=name)
@derived_from(pd.DataFrame)
def std(self, axis=None, skipna=True, ddof=1):
axis = self._validate_axis(axis)
if axis == 1:
return map_partitions(_std, None, self,
token=self._token_prefix + 'std',
axis=axis, skipna=skipna, ddof=ddof)
else:
v = self.var(skipna=skipna, ddof=ddof)
name = self._token_prefix + 'std-finish--%s' % tokenize(self, axis,
skipna, ddof)
return map_partitions(np.sqrt, None, v, token=name)
def quantile(self, q=0.5, axis=0):
""" Approximate row-wise and precise column-wise quantiles of DataFrame
Parameters
----------
q : list/array of floats, default 0.5 (50%)
Iterable of numbers ranging from 0 to 1 for the desired quantiles
axis : {0, 1, 'index', 'columns'} (default 0)
0 or 'index' for row-wise, 1 or 'columns' for column-wise
"""
axis = self._validate_axis(axis)
name = 'quantiles-concat--' + tokenize(self, q, axis)
if axis == 1:
if isinstance(q, list):
# Not supported, the result will have current index as columns
raise ValueError("'q' must be scalar when axis=1 is specified")
return map_partitions(pd.DataFrame.quantile, None, self,
q, axis, token=name)
else:
num = self._get_numeric_data()
quantiles = tuple(quantile(self[c], q) for c in num.columns)
dask = {}
dask = merge(dask, *[q.dask for q in quantiles])
qnames = [(q._name, 0) for q in quantiles]
if isinstance(quantiles[0], Scalar):
dask[(name, 0)] = (pd.Series, (list, qnames), num.columns)
divisions = (min(num.columns), max(num.columns))
return Series(dask, name, None, divisions)
else:
from .multi import _pdconcat
dask[(name, 0)] = (_pdconcat, (list, qnames), 1)
return DataFrame(dask, name, num.columns,
quantiles[0].divisions)
@derived_from(pd.DataFrame)
def describe(self):
name = 'describe--' + tokenize(self)
# currently, only numeric describe is supported
num = self._get_numeric_data()
stats = [num.count(), num.mean(), num.std(), num.min(),
num.quantile([0.25, 0.5, 0.75]), num.max()]
stats_names = [(s._name, 0) for s in stats]
def build_partition(values):
assert len(values) == 6
count, mean, std, min, q, max = values
part1 = self._partition_type([count, mean, std, min],
index=['count', 'mean', 'std', 'min'])
q.index = ['25%', '50%', '75%']
part3 = self._partition_type([max], index=['max'])
return pd.concat([part1, q, part3])
dsk = dict()
dsk[(name, 0)] = (build_partition, (list, stats_names))
dsk = merge(dsk, num.dask, *[s.dask for s in stats])
return self._constructor(dsk, name, num._pd,
divisions=[None, None])
def _cum_agg(self, token, chunk, aggregate, axis, skipna=True,
chunk_kwargs=None):
""" Wrapper for cumulative operation """
axis = self._validate_axis(axis)
if axis == 1:
name = '{0}{1}(axis=1)'.format(self._token_prefix, token)
return self.map_partitions(chunk, token=name, **chunk_kwargs)
else:
# cumulate each partitions
name1 = '{0}{1}-map'.format(self._token_prefix, token)
cumpart = map_partitions(chunk, self._pd, self,
token=name1, **chunk_kwargs)
name2 = '{0}{1}-take-last'.format(self._token_prefix, token)
# cumlast must be a Series or Scalar
cumlast = map_partitions(_take_last, None, cumpart,
skipna, token=name2)
name = '{0}{1}'.format(self._token_prefix, token)
cname = '{0}{1}-cum-last'.format(self._token_prefix, token)
# aggregate cumulated partisions and its previous last element
dask = {}
dask[(name, 0)] = (cumpart._name, 0)
for i in range(1, self.npartitions):
# store each cumulative step to graph to reduce computation
if i == 1:
dask[(cname, i)] = (cumlast._name, i - 1)
else:
# aggregate with previous cumulation results
dask[(cname, i)] = (aggregate, (cname, i - 1),
(cumlast._name, i - 1))
dask[(name, i)] = (aggregate, (cumpart._name, i), (cname, i))
return self._constructor(merge(dask, cumpart.dask, cumlast.dask),
name, chunk(self._pd), self.divisions)
@derived_from(pd.DataFrame)
def cumsum(self, axis=None, skipna=True):
cumsum = lambda x, **kwargs: x.cumsum(**kwargs)
return self._cum_agg('cumsum',
chunk=cumsum,
aggregate=operator.add,
axis=axis, skipna=skipna,
chunk_kwargs=dict(axis=axis, skipna=skipna))
@derived_from(pd.DataFrame)
def cumprod(self, axis=None, skipna=True):
cumprod = lambda x, **kwargs: x.cumprod(**kwargs)
return self._cum_agg('cumprod',
chunk=cumprod,
aggregate=operator.mul,
axis=axis, skipna=skipna,
chunk_kwargs=dict(axis=axis, skipna=skipna))
@derived_from(pd.DataFrame)
def cummax(self, axis=None, skipna=True):
def aggregate(x, y):
if isinstance(x, (pd.Series, pd.DataFrame)):
return x.where((x > y) | x.isnull(), y, axis=x.ndim - 1)
else: # scalar
return x if x > y else y
cummax = lambda x, **kwargs: x.cummax(**kwargs)
return self._cum_agg('cummax',
chunk=cummax,
aggregate=aggregate,
axis=axis, skipna=skipna,
chunk_kwargs=dict(axis=axis, skipna=skipna))
@derived_from(pd.DataFrame)
def cummin(self, axis=None, skipna=True):
def aggregate(x, y):
if isinstance(x, (pd.Series, pd.DataFrame)):
return x.where((x < y) | x.isnull(), y, axis=x.ndim - 1)
else: # scalar
return x if x < y else y
cummin = lambda x, **kwargs: x.cummin(**kwargs)
return self._cum_agg('cummin',
chunk=cummin,
aggregate=aggregate,
axis=axis, skipna=skipna,
chunk_kwargs=dict(axis=axis, skipna=skipna))
@derived_from(pd.DataFrame)
def where(self, cond, other=np.nan):
# cond and other may be dask instance,
# passing map_partitions via keyword will not be aligned
return map_partitions(self._partition_type.where, no_default,
self, cond, other)
@derived_from(pd.DataFrame)
def mask(self, cond, other=np.nan):
return map_partitions(self._partition_type.mask, no_default,
self, cond, other)
@derived_from(pd.Series)
def append(self, other):
# because DataFrame.append will override the method,
# wrap by pd.Series.append docstring
if isinstance(other, (list, dict)):
msg = "append doesn't support list or dict input"
raise NotImplementedError(msg)
if not isinstance(other, _Frame):
from .io import from_pandas
other = from_pandas(other, 1)
from .multi import _append
if self.known_divisions and other.known_divisions:
if self.divisions[-1] < other.divisions[0]:
divisions = self.divisions[:-1] + other.divisions
return _append(self, other, divisions)
else:
msg = ("Unable to append two dataframes to each other with known "
"divisions if those divisions are not ordered. "
"The divisions/index of the second dataframe must be "
"greater than the divisions/index of the first dataframe.")
raise ValueError(msg)
else:
divisions = [None] * (self.npartitions + other.npartitions + 1)
return _append(self, other, divisions)
@classmethod
def _bind_operator_method(cls, name, op):
""" bind operator method like DataFrame.add to this class """
raise NotImplementedError
normalize_token.register((Scalar, _Frame), lambda a: a._name)
class Series(_Frame):
""" Out-of-core Series object
Mimics ``pandas.Series``.
Parameters
----------
dsk: dict
The dask graph to compute this Series
_name: str
The key prefix that specifies which keys in the dask comprise this
particular Series
name: scalar or None
Series name. This metadata aids usability
divisions: tuple of index values
Values along which we partition our blocks on the index
See Also
--------
dask.dataframe.DataFrame
"""
_partition_type = pd.Series
_token_prefix = 'series-'
def __new__(cls, dsk, _name, name, divisions):
result = object.__new__(cls)
result.dask = dsk
result._name = _name
result._pd, result._known_dtype = cls._build_pd(name)
result.divisions = tuple(divisions)
return result
@property
def _args(self):
return (self.dask, self._name, self.name, self.divisions)
@property
def _constructor_sliced(self):
return Scalar
@property
def _constructor(self):
return Series
@property
def name(self):
return self._pd.name
@name.setter
def name(self, name):
self._pd.name = name
renamed = _rename_dask(self, name)
# update myself
self.dask.update(renamed.dask)
self._name = renamed._name
@property
def ndim(self):
""" Return dimensionality """
return 1
@property
def dtype(self):
""" Return data type """
if self._known_dtype:
return self._pd.dtype
else:
self._pd, self._known_dtype = self._build_pd(self.head())
return self._pd.dtype
def __getattr__(self, key):
if key == 'cat':
# If unknown dtype, need to infer from head.
if not self._known_dtype:
self.dtype
return self._pd.cat
raise AttributeError("'Series' object has no attribute %r" % key)
@property
def column_info(self):
""" Return Series.name """
warnings.warn('column_info is deprecated, use name')
return self.name
@property