/
core.py
1517 lines (1217 loc) · 49.7 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 division
from itertools import count
from math import sqrt
from functools import wraps, reduce
from collections import Iterable
import bisect
import uuid
from hashlib import md5
from toolz import merge, partial, first, partition
from operator import getitem, setitem
from datetime import datetime
import pandas as pd
import numpy as np
import operator
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 unicode, apply
from ..utils import repr_long_list, IndexCallable, pseudorandom
from .utils import shard_df_on_index, tokenize_dataframe
from ..base import Base, compute
no_default = '__no_default__'
def tokenize(obj):
""" Deterministic token
>>> tokenize([1, 2, '3'])
'b9e8c0d38fb40e66dc4fd00adc3c6553'
>>> tokenize('Hello') == tokenize('Hello')
True
"""
return md5(str(obj).encode()).hexdigest()
def _concat(args):
""" Generic concat operation """
if not args:
return args
if isinstance(first(core.flatten(args)), np.ndarray):
return da.core.concatenate3(args)
if len(args) == 1:
return args[0]
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)]
result = pd.concat(map(pd.Series, args))
result = type(args[0])(result.values)
result.name = args[0].name
return result
return args
tokens = ('-%d' % i for i in count(1))
def optimize(dsk, keys):
from .optimize import optimize
return optimize(dsk, keys)
def finalize(self, 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(finalize)
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
@property
def _args(self):
return (self.dask, self._name)
def _keys(self):
return [(self._name, 0)]
class _Frame(Base):
""" Superclass for DataFrame and Series """
_optimize = staticmethod(optimize)
_default_get = staticmethod(threaded.get)
_finalize = staticmethod(finalize)
# 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 len(self.divisions) - 1
def _keys(self):
return [(self._name, i) for i in range(self.npartitions)]
@property
def index(self):
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, None, self.divisions)
@property
def known_divisions(self):
return len(self.divisions) > 0 and self.divisions[0] is not None
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.column_info, self.divisions)
@wraps(pd.DataFrame.drop_duplicates)
def drop_duplicates(self):
chunk = lambda s: s.drop_duplicates()
return aca(self, chunk=chunk, aggregate=chunk, columns=self.columns,
token='drop-duplicates')
def __len__(self):
return reduction(self, len, np.sum).compute()
def map_partitions(self, func, columns=no_default):
""" 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``.
>>> df.map_partitions(lambda df: df.x + 1, columns='x') # doctest: +SKIP
>>> df.map_partitions(lambda df: df.head(), columns=df.columns) # doctest: +SKIP
Parameters
----------
column_info: tuple or string
Column names or name of the output.
Defaults to names of the input.
"""
if columns == no_default:
columns = self.column_info
return map_partitions(func, columns, self)
def random_split(self, p, seed=None):
""" Pseudorandomly split dataframe into different pieces row-wise
50/50 split
>>> a, b = df.random_split([0.5, 0.5]) # doctest: +SKIP
80/10/10 split, consistent seed
>>> a, b, c = df.random_split([0.8, 0.1, 0.1], seed=123) # doctest: +SKIP
"""
seeds = np.random.RandomState(seed).randint(0, np.iinfo(np.int32).max,
self.npartitions)
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.column_info,
self.divisions)
for i, dsk in enumerate(dsks)]
def head(self, n=5, compute=True):
""" First n rows of the dataset
Caveat, the 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.column_info, 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.column_info, 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)
elif isinstance(ind, slice):
return self._loc_slice(ind)
else:
return self._loc_element(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.columns, self, ind, token='loc-series')
def _loc_element(self, ind):
name = 'loc-element-%s-%s' % (str(ind), self._name)
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], (self._name, part))}
return self._constructor_sliced(merge(self.dask, dsk), name,
self.column_info, [ind, ind])
def _loc_slice(self, ind):
name = 'loc-slice-%s-%s' % (str(ind), self._name)
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.column_info, divisions)
@property
def loc(self):
return IndexCallable(self._loc)
@property
def iloc(self):
# not implemented because of performance concerns.
# see https://github.com/ContinuumIO/dask/pull/507
raise AttributeError("Dask Dataframe does not support iloc")
def repartition(self, divisions):
""" Repartition dataframe along new divisions
>>> df = df.repartition([0, 5, 10, 20]) # doctest: +SKIP
"""
return repartition(self, divisions)
def __getstate__(self):
return self.__dict__
def __setstate__(self, dict):
self.__dict__ = dict
@wraps(pd.Series.fillna)
def fillna(self, value):
func = getattr(self._partition_type, 'fillna')
return map_partitions(func, self.column_info, self, value)
def sample(self, frac, random_state=None):
""" Random sample of items
This only implements the ``frac`` option from pandas.
See Also:
pd.DataFrame.sample
"""
if random_state is None:
random_state = np.random.randint(np.iinfo(np.int32).max)
name = 'sample-' + tokenize((self._name, frac, random_state))
func = getattr(self._partition_type, 'sample')
if not isinstance(random_state, np.random.RandomState):
random_state = np.random.RandomState(random_state)
seeds = random_state.randint(np.iinfo(np.int32).max,
size=self.npartitions)
dsk = dict(((name, i),
(apply, func, (tuple, [(self._name, i)]),
{'frac': frac, 'random_state': seed}))
for i, seed in zip(range(self.npartitions), seeds))
return self._constructor(merge(self.dask, dsk), name,
self.column_info, self.divisions)
@wraps(pd.DataFrame.to_hdf)
def to_hdf(self, path_or_buf, key, mode='a', append=False, complevel=0,
complib=None, fletcher32=False, **kwargs):
from .io import to_hdf
return to_hdf(self, path_or_buf, key, mode, append, complevel, complib,
fletcher32, **kwargs)
@property
def _elemwise_cols(self):
"""passed to elemwise ops, None for Series, columns for DataFrame"""
return None
def __abs__(self):
return elemwise(operator.abs, self, columns=self._elemwise_cols)
def __add__(self, other):
return elemwise(operator.add, self, other, columns=self._elemwise_cols)
def __radd__(self, other):
return elemwise(operator.add, other, self, columns=self._elemwise_cols)
def __and__(self, other):
return elemwise(operator.and_, self, other, columns=self._elemwise_cols)
def __rand__(self, other):
return elemwise(operator.and_, other, self, columns=self._elemwise_cols)
def __div__(self, other):
return elemwise(operator.div, self, other, columns=self._elemwise_cols)
def __rdiv__(self, other):
return elemwise(operator.div, other, self, columns=self._elemwise_cols)
def __eq__(self, other):
return elemwise(operator.eq, self, other, columns=self._elemwise_cols)
def __gt__(self, other):
return elemwise(operator.gt, self, other, columns=self._elemwise_cols)
def __ge__(self, other):
return elemwise(operator.ge, self, other, columns=self._elemwise_cols)
def __invert__(self):
return elemwise(operator.inv, self, columns=self._elemwise_cols)
def __lt__(self, other):
return elemwise(operator.lt, self, other, columns=self._elemwise_cols)
def __le__(self, other):
return elemwise(operator.le, self, other, columns=self._elemwise_cols)
def __mod__(self, other):
return elemwise(operator.mod, self, other, columns=self._elemwise_cols)
def __rmod__(self, other):
return elemwise(operator.mod, other, self, columns=self._elemwise_cols)
def __mul__(self, other):
return elemwise(operator.mul, self, other, columns=self._elemwise_cols)
def __rmul__(self, other):
return elemwise(operator.mul, other, self, columns=self._elemwise_cols)
def __ne__(self, other):
return elemwise(operator.ne, self, other, columns=self._elemwise_cols)
def __neg__(self):
return elemwise(operator.neg, self, columns=self._elemwise_cols)
def __or__(self, other):
return elemwise(operator.or_, self, other, columns=self._elemwise_cols)
def __ror__(self, other):
return elemwise(operator.or_, other, self, columns=self._elemwise_cols)
def __pow__(self, other):
return elemwise(operator.pow, self, other, columns=self._elemwise_cols)
def __rpow__(self, other):
return elemwise(operator.pow, other, self, columns=self._elemwise_cols)
def __sub__(self, other):
return elemwise(operator.sub, self, other, columns=self._elemwise_cols)
def __rsub__(self, other):
return elemwise(operator.sub, other, self, columns=self._elemwise_cols)
def __truediv__(self, other):
return elemwise(operator.truediv, self, other, columns=self._elemwise_cols)
def __rtruediv__(self, other):
return elemwise(operator.truediv, other, self, columns=self._elemwise_cols)
def __floordiv__(self, other):
return elemwise(operator.floordiv, self, other, columns=self._elemwise_cols)
def __rfloordiv__(self, other):
return elemwise(operator.floordiv, other, self, columns=self._elemwise_cols)
def __xor__(self, other):
return elemwise(operator.xor, self, other, columns=self._elemwise_cols)
def __rxor__(self, other):
return elemwise(operator.xor, other, self, columns=self._elemwise_cols)
class Series(_Frame):
""" Out-of-core Series object
Mimics ``pandas.Series``.
See Also
--------
dask.dataframe.DataFrame
"""
_partition_type = pd.Series
def __init__(self, dsk, _name, name, divisions):
self.dask = dsk
self._name = _name
self.name = name
self.divisions = tuple(divisions)
self.dt = DatetimeAccessor(self)
self.str = StringAccessor(self)
@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 dtype(self):
return self.head().dtype
@property
def column_info(self):
return self.name
@property
def columns(self):
return (self.name,)
def __repr__(self):
return ("dd.%s<%s, divisions=%s>" %
(self.__class__.__name__, self._name,
repr_long_list(self.divisions)))
def quantile(self, q):
""" Approximate quantiles of column
q : list/array of floats
Iterable of numbers ranging from 0 to 1 for the desired quantiles
"""
# pandas uses quantile in [0, 1]
# numpy / everyone else uses [0, 100]
return quantile(self, np.asarray(q) * 100)
def quantiles(self, *args, **kwargs):
raise NotImplementedError("This has moved to quantile to match the Pandas API\n"
"Also, quantiles will now be specified on the range "
"[0, 1], not [0, 100]")
def __getitem__(self, key):
if isinstance(key, Series) and self.divisions == key.divisions:
name = 'series-index-%s[%s]' % (self._name, key._name)
dsk = dict(((name, i), (operator.getitem, (self._name, i),
(key._name, i)))
for i in range(self.npartitions))
return Series(merge(self.dask, key.dask, dsk), name,
self.name, self.divisions)
raise NotImplementedError()
@wraps(pd.Series.sum)
def sum(self):
return reduction(self, pd.Series.sum, np.sum)
@wraps(pd.Series.max)
def max(self):
return reduction(self, pd.Series.max, np.max)
@wraps(pd.Series.min)
def min(self):
return reduction(self, pd.Series.min, np.min)
@wraps(pd.Series.count)
def count(self):
return reduction(self, pd.Series.count, np.sum)
@wraps(pd.Series.nunique)
def nunique(self):
return self.drop_duplicates().count()
@wraps(pd.Series.mean)
def mean(self):
def chunk(ser):
return (ser.sum(), ser.count())
def agg(seq):
sums, counts = list(zip(*seq))
return 1.0 * sum(sums) / sum(counts)
return reduction(self, chunk, agg, 'mean')
@wraps(pd.Series.var)
def var(self, ddof=1):
def chunk(ser):
return (ser.sum(), (ser**2).sum(), ser.count())
def agg(seq):
x, x2, n = list(zip(*seq))
x = float(sum(x))
x2 = float(sum(x2))
n = sum(n)
result = (x2 / n) - (x / n)**2
if ddof:
result = result * n / (n - ddof)
return result
return reduction(self, chunk, agg, token=('var', ddof))
@wraps(pd.Series.std)
def std(self, ddof=1):
name = '%s.std(ddof=%d)' % (self._name, ddof)
df = self.var(ddof=ddof)
dsk = {(name, 0): (sqrt, (df._name, 0))}
return Scalar(merge(df.dask, dsk), name)
@wraps(pd.Series.value_counts)
def value_counts(self):
chunk = lambda s: s.value_counts()
agg = lambda s: s.groupby(level=0).sum().sort(inplace=False, ascending=False)
return aca(self, chunk=chunk, aggregate=agg, columns=self.columns,
token='value-counts')
@wraps(pd.Series.nlargest)
def nlargest(self, n=5):
f = lambda s: s.nlargest(n)
return aca(self, f, f, columns=self.columns, token=('nlargest', n))
@wraps(pd.Series.isin)
def isin(self, other):
return elemwise(pd.Series.isin, self, other)
@wraps(pd.Series.map)
def map(self, arg, na_action=None):
return elemwise(pd.Series.map, self, arg, na_action, name=self.name)
@wraps(pd.Series.astype)
def astype(self, dtype):
return map_partitions(pd.Series.astype, self.name, self, dtype)
@wraps(pd.Series.dropna)
def dropna(self):
return map_partitions(pd.Series.dropna, self.name, self)
@wraps(pd.Series.between)
def between(self, left, right, inclusive=True):
return map_partitions(pd.Series.between, self.name, self, left, right,
inclusive)
@wraps(pd.Series.clip)
def clip(self, lower=None, upper=None):
return map_partitions(pd.Series.clip, self.name, self, lower, upper)
@wraps(pd.Series.notnull)
def notnull(self):
return map_partitions(pd.Series.notnull, self.name, self)
class Index(Series):
@property
def index(self):
msg = "'{0}' object has no attribute 'index'"
raise AttributeError(msg.format(self.__class__.__name__))
@property
def _constructor(self):
return Index
def nunique(self):
return self.drop_duplicates().count()
def count(self):
f = lambda x: pd.notnull(x).sum()
return reduction(self, f, np.sum)
class DataFrame(_Frame):
"""
Implements out-of-core DataFrame as a sequence of pandas DataFrames
This is a work in progress. It is buggy and far from complete.
Please do not use it yet.
Parameters
----------
dask: dict
The dask graph to compute this Dataframe
name: str
The key prefix that specifies which keys in the dask comprise this
particular DataFrame
columns: list of strings
Column names. This metadata aids usability
divisions: tuple of index values
Values along which we partition our blocks on the index
"""
_partition_type = pd.DataFrame
def __init__(self, dask, name, columns, divisions):
self.dask = dask
self._name = name
self.columns = tuple(columns)
self.divisions = tuple(divisions)
@property
def _args(self):
return (self.dask, self._name, self.columns, self.divisions)
@property
def _constructor_sliced(self):
return Series
@property
def _constructor(self):
return DataFrame
def __getitem__(self, key):
if isinstance(key, (str, unicode)):
name = self._name + '.' + key
if key in self.columns:
dsk = dict(((name, i), (operator.getitem, (self._name, i), key))
for i in range(self.npartitions))
return self._constructor_sliced(merge(self.dask, dsk), name,
key, self.divisions)
if isinstance(key, list):
name = '%s[%s]' % (self._name, str(key))
if all(k in self.columns for k in key):
dsk = dict(((name, i), (operator.getitem,
(self._name, i),
(list, key)))
for i in range(self.npartitions))
return self._constructor(merge(self.dask, dsk), name,
key, self.divisions)
if isinstance(key, Series) and self.divisions == key.divisions:
name = 'series-slice-%s[%s]' % (self._name, key._name)
dsk = dict(((name, i), (operator.getitem, (self._name, i),
(key._name, i)))
for i in range(self.npartitions))
return self._constructor(merge(self.dask, key.dask, dsk), name,
self.columns, self.divisions)
raise NotImplementedError()
def __getattr__(self, key):
try:
return object.__getattribute__(self, key)
except AttributeError as e:
try:
return self[key]
except NotImplementedError:
raise e
def __dir__(self):
return sorted(set(dir(type(self)) + list(self.__dict__) +
list(self.columns)))
def __repr__(self):
return ("dd.DataFrame<%s, divisions=%s>" %
(self._name, repr_long_list(self.divisions)))
@property
def dtypes(self):
return self._get(self.dask, self._keys()[0]).dtypes
def set_index(self, other, **kwargs):
return set_index(self, other, **kwargs)
def set_partition(self, column, divisions, **kwargs):
""" Set explicit divisions for new column index
>>> df2 = df.set_partition('new-index-column', divisions=[10, 20, 50]) # doctest: +SKIP
See also:
set_index
"""
return set_partition(self, column, divisions, **kwargs)
@property
def column_info(self):
return self.columns
def groupby(self, key, **kwargs):
return GroupBy(self, key, **kwargs)
def categorize(self, columns=None, **kwargs):
return categorize(self, columns, **kwargs)
@wraps(pd.DataFrame.assign)
def assign(self, **kwargs):
pairs = list(sum(kwargs.items(), ()))
# Figure out columns of the output
df = pd.DataFrame(columns=self.columns)
df2 = df.assign(**dict((k, []) for k in kwargs))
return elemwise(_assign, self, *pairs, columns=list(df2.columns))
def query(self, expr, **kwargs):
""" Blocked version of pd.DataFrame.query
This is like the sequential version except that this will also happen
in many threads. This may conflict with ``numexpr`` which will use
multiple threads itself. We recommend that you set numexpr to use a
single thread
import numexpr
numexpr.set_nthreads(1)
The original docstring follows below:\n
""" + pd.DataFrame.query.__doc__
name = '%s.query(%s)' % (self._name, expr)
if kwargs:
name = name + '--' + tokenize(sorted(kwargs.items()))
dsk = dict(((name, i), (apply, pd.DataFrame.query,
((self._name, i), (expr,), kwargs)))
for i in range(self.npartitions))
else:
dsk = dict(((name, i), (pd.DataFrame.query, (self._name, i), expr))
for i in range(self.npartitions))
return self._constructor(merge(dsk, self.dask), name,
self.columns, self.divisions)
@wraps(pd.DataFrame.dropna)
def dropna(self, how='any', subset=None):
def f(df, how=how, subset=subset):
return df.dropna(how=how, subset=subset)
return map_partitions(f, self.columns, self)
def to_castra(self, fn=None, categories=None, compute=True):
""" Write DataFrame to Castra on-disk store
See https://github.com/blosc/castra for details
See Also:
Castra.to_dask
"""
from .io import to_castra
return to_castra(self, fn, categories, compute=compute)
@wraps(pd.DataFrame.to_csv)
def to_csv(self, filename, **kwargs):
from .io import to_csv
return to_csv(self, filename, **kwargs)
@property
def _elemwise_cols(self):
return self.columns
@wraps(pd.DataFrame.drop)
def drop(self, labels, axis=0):
if axis != 1:
raise NotImplementedError("Drop currently only works for axis=1")
columns = list(pd.DataFrame(columns=self.columns)
.drop(labels, axis=axis)
.columns)
return elemwise(pd.DataFrame.drop, self, labels, axis, columns=columns)
def _assign(df, *pairs):
kwargs = dict(partition(2, pairs))
return df.assign(**kwargs)
def _partition_of_index_value(divisions, val):
""" In which partition does this value lie?
>>> _partition_of_index_value([0, 5, 10], 3)
0
>>> _partition_of_index_value([0, 5, 10], 8)
1
>>> _partition_of_index_value([0, 5, 10], 100)
1
>>> _partition_of_index_value([0, 5, 10], 5) # left-inclusive divisions
1
"""
if divisions[0] is None:
raise ValueError(
"Can not use loc on DataFrame without known divisions")
val = _coerce_loc_index(divisions, val)
i = bisect.bisect_right(divisions, val)
return min(len(divisions) - 2, max(0, i - 1))
def _loc(df, start, stop, include_right_boundary=True):
"""
>>> df = pd.DataFrame({'x': [10, 20, 30, 40, 50]}, index=[1, 2, 2, 3, 4])
>>> _loc(df, 2, None)
x
2 20
2 30
3 40
4 50
>>> _loc(df, 1, 3)
x
1 10
2 20
2 30
3 40
>>> _loc(df, 1, 3, include_right_boundary=False)
x
1 10
2 20
2 30
"""
result = df.loc[slice(start, stop)]
if not include_right_boundary:
# result = df[df.index != stop]
result = result.iloc[:result.index.get_slice_bound(stop, 'left',
result.index.inferred_type)]
return result
def _coerce_loc_index(divisions, o):
""" Transform values to be comparable against divisions
This is particularly valuable to use with pandas datetimes
"""
if divisions and isinstance(divisions[0], datetime):
return pd.Timestamp(o)
if divisions and isinstance(divisions[0], np.datetime64):
return np.datetime64(o)
return o
def consistent_name(names):
""" New name for series in elementwise operation
If all truthy names are the same, choose that one, otherwise, choose None
"""
allnames = set()
for name in names:
if name is None:
continue
if isinstance(name, (tuple, list)):
allnames.update(name)
else:
allnames.add(name)
if len(allnames) == 1:
return first(allnames)
else:
return None
def elemwise(op, *args, **kwargs):
""" Elementwise operation for dask.Dataframes """
columns = kwargs.get('columns', None)
name = kwargs.get('name', None)
token = (op,
[arg._name if isinstance(arg, _Frame) else arg for arg in args],
sorted(kwargs.items(), key=lambda kv: str(kv[0])))
_name = 'elemwise-' + tokenize(token)
dfs = [arg for arg in args if isinstance(arg, _Frame)]
other = [(i, arg) for i, arg in enumerate(args)
if not isinstance(arg, _Frame)]
if other:
op2 = partial_by_order(op, other)
else:
op2 = op
if not all(df.divisions == dfs[0].divisions for df in dfs):
msg = 'All dask.Dataframe and dask.Series must have same divisions'
raise ValueError(msg)
if not all(df.npartitions == dfs[0].npartitions for df in dfs):
msg = 'All dask.Dataframe and dask.Series must have same npartitions'
raise ValueError(msg)
dsk = dict(((_name, i), (op2,) + frs)
for i, frs in enumerate(zip(*[df._keys() for df in dfs])))
if columns is not None:
return DataFrame(merge(dsk, *[df.dask for df in dfs]),
_name, columns, dfs[0].divisions)
else:
column_name = name or consistent_name(n for df in dfs
for n in df.columns)
return Series(merge(dsk, *[df.dask for df in dfs]),
_name, column_name, dfs[0].divisions)
def remove_empties(seq):
""" Remove items of length 0
>>> remove_empties([1, 2, ('empty', np.nan), 4, 5])
[1, 2, 4, 5]
>>> remove_empties([('empty', np.nan)])
[nan]
>>> remove_empties([])
[]
"""
if not seq:
return seq
seq2 = [x for x in seq
if not (isinstance(x, tuple) and x and x[0] == 'empty')]
if seq2:
return seq2
else:
return [seq[0][1]]
def empty_safe(func, arg):
"""
>>> empty_safe(sum, [1, 2, 3])
6
>>> empty_safe(sum, [])
('empty', 0)
"""
if len(arg) == 0:
return ('empty', func(arg))
else:
return func(arg)
def reduction(x, chunk, aggregate, token=None):
""" General version of reductions
>>> reduction(my_frame, np.sum, np.sum) # doctest: +SKIP
"""
token = (x._name, (token or (chunk, aggregate)))
token = tokenize(token)
a = 'reduction-chunk-' + token
dsk = dict(((a, i), (empty_safe, chunk, (x._name, i)))
for i in range(x.npartitions))
b = 'reduction-aggregation-' + token
dsk2 = {(b, 0): (aggregate, (remove_empties,
[(a,i) for i in range(x.npartitions)]))}
return Scalar(merge(x.dask, dsk, dsk2), b)
def concat(dfs):
""" Concatenate dataframes along rows
Currently only supports unknown divisions
"""
if any(df.known_divisions for df in dfs):