This repository has been archived by the owner on May 8, 2024. It is now read-only.
/
rdd.py
2430 lines (2020 loc) · 81.5 KB
/
rdd.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
from __future__ import print_function
import sys
import os, os.path
import time
import socket
import csv
import itertools
import collections
import math
import six
from six.moves import filter, map, range, zip, cPickle
import random
import bz2
import gzip
import zlib
from copy import copy
import shutil
import heapq
import struct
import tempfile
try:
from cStringIO import StringIO
BytesIO = StringIO
except ImportError:
from six import BytesIO, StringIO
try:
from cPickle import Pickler, Unpickler
except ImportError:
from pickle import Pickler, Unpickler
import dpark.conf
from dpark.dependency import *
from dpark.utils import (
spawn, chain, mkdir_p, recursion_limit_breaker, atomic_file,
AbortFileReplacement, portable_hash,
masked_crc32c
)
from dpark.utils import DparkUserFatalError
from dpark.utils.log import get_logger
from dpark.utils.frame import Scope, func_info
from dpark.shuffle import SortShuffleFetcher, Merger
from dpark.env import env
from dpark.file_manager import open_file, CHUNKSIZE
from dpark.utils.beansdb import BeansdbReader, BeansdbWriter
from contextlib import closing
from functools import reduce
if not six.PY2:
from io import TextIOWrapper
logger = get_logger(__name__)
class Split(object):
def __init__(self, idx):
self.index = idx
def cached(func):
def getstate(self):
d = getattr(self, '_pickle_cache', None)
if d is None:
d = func(self)
self._pickle_cache = d
return d
return getstate
class RDD(object):
def __init__(self, ctx):
self.ctx = ctx
self.id = RDD.new_id()
self._splits = []
self._dependencies = []
self.aggregator = None
self._partitioner = None
self.shouldCache = False
self.checkpoint_path = None
self._checkpoint_rdd = None
ctx.init()
self.mem = ctx.options.mem
self.cpus = 0
self.gpus = 0
self._preferred_locs = {}
self.repr_name = '<%s>' % (self.__class__.__name__,)
self.scope = Scope.get(self.__class__.__name__)
self.rddconf = None
self.lineage = self.scope.stackhash
self._dep_lineage_counts = None # map "dep rdd id with uniq lineages" to their counts
self.err_ratio = ctx.options.err
self.allow_err = self.err_ratio > 1e-8
self.exc_info = (None, None, None)
nextId = 0
@classmethod
def new_id(cls):
cls.nextId += 1
return cls.nextId
@cached
def __getstate__(self):
d = dict(self.__dict__)
d.pop('_dependencies', None)
d.pop('_splits', None)
d.pop('_preferred_locs', None)
d.pop('_dep_lineage_counts', None)
d.pop('ctx', None)
d['_split_size'] = len(self.splits)
return d
@property
def dep_lineage_counts(self):
if self._dep_lineage_counts is None:
lineages = collections.defaultdict(list)
for dep in self._dependencies:
rdd = dep.rdd
lineages[rdd.lineage].append(rdd)
self._dep_lineage_counts = dict([(rs[0].id, len(rs)) for rs in lineages.values()])
return self._dep_lineage_counts
@property
def params(self):
return None
def __len__(self):
if hasattr(self, '_split_size'):
return self._split_size
return len(self.splits)
def __repr__(self):
return self.repr_name
def __getslice__(self, i, j):
return SliceRDD(self, i, j)
def __getitem__(self, idx):
if isinstance(idx, slice) and idx.step is None:
start = idx.start or 0
stop = idx.stop or sys.maxsize
return self.__getslice__(start, stop)
raise ValueError('Unsupported slicing!')
def mergeSplit(self, splitSize=None, numSplits=None):
return MergedRDD(self, splitSize, numSplits)
@property
def splits(self):
if self._checkpoint_rdd:
return self._checkpoint_rdd.splits
return self._splits
@property
def dependencies(self):
if self._checkpoint_rdd:
return self._checkpoint_rdd.dependencies
return self._dependencies
def compute(self, split):
raise NotImplementedError
@property
def partitioner(self):
return self._partitioner
@property
def ui_label(self):
return "{}[{}]".format(self.__class__.__name__, len(self))
def cache(self):
self.shouldCache = True
self._pickle_cache = None # clear pickle cache
return self
def preferredLocations(self, split):
if self._checkpoint_rdd:
return self._checkpoint_rdd._preferred_locs.get(split, [])
if self.shouldCache:
locs = env.cacheTrackerServer.getCachedLocs(self.id, split.index)
if locs:
return locs
return self._preferred_locs.get(split, [])
def checkpoint(self, path=None):
if path is None:
path = self.ctx.options.checkpoint_dir
if path:
ident = '%d_%x' % (self.id, hash(str(self)))
path = os.path.join(path, ident)
mkdir_p(path)
self.checkpoint_path = path
else:
logger.warning('No checkpoint will be saved without checkpoint_dir,'
'please re-run with --checkpoint-dir to enable checkpoint')
return self
def _clear_dependencies(self):
self._dependencies = []
self._splits = []
def _do_checkpoint(self):
if self.checkpoint_path:
if not self._checkpoint_rdd:
_generated = list(map(int, CheckpointRDD.generated_files(self.checkpoint_path)))
if len(_generated) != len(self):
missing = [sp.index for sp in self.splits if sp.index not in _generated]
sum(self.ctx.runJob(self, lambda x: list(x), missing), [])
self._pickle_cache = None
self._checkpoint_rdd = CheckpointRDD(self.ctx, self.checkpoint_path)
self._clear_dependencies()
return False
return True
@recursion_limit_breaker
def iterator(self, split):
def _compute(rdd, split_):
if self.shouldCache:
return env.cacheTracker.getOrCompute(rdd, split_)
else:
return rdd.compute(split_)
if self.checkpoint_path:
if self._checkpoint_rdd is None:
p = os.path.join(self.checkpoint_path, str(split.index))
v = list(self.compute(split))
with atomic_file(p) as f:
f.write(cPickle.dumps(v, -1))
return v
else:
return _compute(self._checkpoint_rdd, split)
return _compute(self, split)
def set_rddconf(self, rddconf):
if rddconf is None:
rddconf = dpark.conf.rddconf()
elif rddconf.sort_merge:
num_map = self._num_stream_need()
limit = dpark.conf.MAX_OPEN_FILE
if num_map > limit:
logger.warning("%s: too many split (%d > %d) in parents stages,"
" sort merge will use disk",
self.scope.api_callsite, num_map, limit)
rddconf.disk_merge = True
self.rddconf = rddconf
def map(self, f):
return MappedRDD(self, f)
def flatMap(self, f):
return FlatMappedRDD(self, f)
def filter(self, f):
return FilteredRDD(self, f)
def sample(self, faction, withReplacement=False, seed=12345):
return SampleRDD(self, faction, withReplacement, seed)
def union(self, *args):
return UnionRDD(self.ctx, [self] + list(args))
def sort(self, key=lambda x: x, reverse=False, numSplits=None, taskMemory=None, rddconf=None):
if not len(self):
return self
if len(self) == 1:
return self.mapPartitions(lambda it: sorted(it, key=key, reverse=reverse))
if numSplits is None:
numSplits = min(self.ctx.defaultMinSplits, len(self))
n = max(numSplits * 10 // len(self), 1)
samples = self.mapPartitions(lambda x: itertools.islice(x, n)).map(key).collect()
keys = sorted(samples, reverse=reverse)[5::10][:numSplits - 1]
parter = RangePartitioner(keys, reverse=reverse)
aggr = MergeAggregator()
parted = ShuffledRDD(self.map(lambda x: (key(x), x)), aggr, parter, taskMemory, rddconf=rddconf).flatMap(
lambda x_y: x_y[1])
return parted.mapPartitions(lambda x: sorted(x, key=key, reverse=reverse))
def glom(self):
return GlommedRDD(self)
def cartesian(self, other, taskMemory=None, cacheMemory=None):
return CartesianRDD(self, other, taskMemory=taskMemory, cacheMemory=cacheMemory)
def zipWith(self, other):
return ZippedRDD(self.ctx, [self, other])
def groupBy(self, f, numSplits=None, rddconf=None):
if numSplits is None:
numSplits = min(self.ctx.defaultMinSplits, len(self))
return self.map(lambda x: (f(x), x)).groupByKey(numSplits, rddconf=rddconf)
def pipe(self, command, quiet=False):
if isinstance(command, str):
command = command.split(' ')
return PipedRDD(self, command, quiet)
def fromCsv(self, dialect='excel'):
return CSVReaderRDD(self, dialect)
def mapPartitions(self, f):
return MapPartitionsRDD(self, f)
mapPartition = mapPartitions
def foreach(self, f):
def mf(it):
for i in it:
f(i)
list(self.ctx.runJob(self, mf))
def foreachPartition(self, f):
list(self.ctx.runJob(self, f))
def enumeratePartition(self):
return EnumeratePartitionsRDD(self, lambda x, it: map(lambda y: (x, y), it))
def enumerate(self):
"""
enumerate this RDD.
>>> dpark.parallelize(["a", "b", "c", "d"], 3).enumerate().collect()
[(0, 'a'), (1, 'b'), (2, 'c'), (3, 'd')]
"""
starts = [0]
if len(self) > 1:
nums = self.mapPartitions(lambda it: [sum(1 for _ in it)]).collect()
for i in range(len(nums) - 1):
starts.append(starts[-1] + nums[i])
return EnumeratePartitionsRDD(self, lambda x, it: enumerate(it, starts[x]))
def collect(self):
return sum(self.ctx.runJob(self, lambda x: list(x)), [])
def __iter__(self):
return chain(self.ctx.runJob(self, lambda x: list(x)))
def reduce(self, f):
def reducePartition(it):
logger = get_logger(__name__)
if not self.allow_err:
try:
return [reduce(f, it)]
except TypeError as e:
empty_msg = 'reduce() of empty sequence with no initial value'
if e.message == empty_msg:
return []
else:
raise e
s = None
total, err = 0, 0
for v in it:
try:
total += 1
if s is None:
s = v
else:
s = f(s, v)
except Exception as e:
logger.warning("skip bad record %s: %s", v, e)
err += 1
self.check_err_rate(total, err, False)
self.check_err_rate(total, err, True)
return [s] if s is not None else []
return reduce(f, chain(self.ctx.runJob(self, reducePartition)))
def uniq(self, numSplits=None, taskMemory=None, rddconf=None):
g = self.map(lambda x: (x, None)).reduceByKey(lambda x, y: None, numSplits, taskMemory, rddconf)
return g.map(lambda x_y1: x_y1[0])
def top(self, n=10, key=None, reverse=False):
if reverse:
def topk(it):
return heapq.nsmallest(n, it, key)
else:
def topk(it):
return heapq.nlargest(n, it, key)
return topk(sum(self.ctx.runJob(self, topk), []))
def hot(self, n=10, numSplits=None, taskMemory=None, rddconf=None):
st = self.map(lambda x: (x, 1)).reduceByKey(lambda x, y: x + y, numSplits, taskMemory, rddconf=rddconf)
return st.top(n, key=lambda x: x[1])
def fold(self, zero, f):
'''Aggregate the elements of each partition, and then the
results for all the partitions, using a given associative
function and a neutral "zero value". The function op(t1, t2)
is allowed to modify t1 and return it as its result value to
avoid object allocation; however, it should not modify t2.'''
return reduce(f,
self.ctx.runJob(self, lambda x: reduce(f, x, copy(zero))),
zero)
def aggregate(self, zero, seqOp, combOp):
'''Aggregate the elements of each partition, and then the
results for all the partitions, using given combine functions
and a neutral "zero value". This function can return a
different result type, U, than the type of this RDD, T. Thus,
we need one operation for merging a T into an U (seqOp(U, T))
and one operation for merging two U's (combOp(U, U)). Both of
these functions are allowed to modify and return their first
argument instead of creating a new U to avoid memory
allocation.'''
return reduce(combOp,
self.ctx.runJob(self, lambda x: reduce(seqOp, x, copy(zero))),
zero)
def count(self):
return sum(self.ctx.runJob(self, lambda x: sum(1 for i in x)))
def toList(self):
return self.collect()
def take(self, n):
if n == 0:
return []
r = []
p = 0
n_splits = 1
last = 0
while len(r) < n and p < len(self):
left = n - len(r)
if p > 0:
# assume we has only 10 but take(11), and get the 10 early, we need to run remaining splits fast.
n_splits = 2 * n_splits
if last > 0:
n_splits = min(int(math.ceil(float(left) * p / len(r))), n_splits)
max_ = min((p + n_splits), len(self))
splits = list(range(p, max_))
logger.info("try to TAKE remaining %d of %d results, from %d/%d splits: [%d, %d)",
left, n, len(splits), len(self), p, max_)
list_list = list(self.ctx.runJob(self, lambda x: list(itertools.islice(x, left)), splits, True))
res = [item for sublist in list_list for item in sublist][:left]
last = len(res)
if res:
r.extend(res)
p = max_
return r
def first(self):
r = self.take(1)
if r: return r[0]
def saveAsTextFile(self, path, ext='', overwrite=True, compress=False):
return OutputTextFileRDD(self, path, ext, overwrite, compress=compress).collect()
def saveAsTFRecordsFile(self, path, ext='', overwrite=True, compress=False):
return OutputTfrecordstFileRDD(self, path, ext, overwrite, compress=compress).collect()
def saveAsTextFileByKey(self, path, ext='', overwrite=True, compress=False):
return MultiOutputTextFileRDD(self, path, ext, overwrite, compress=compress).collect()
def saveAsCSVFile(self, path, dialect='excel', overwrite=True, compress=False):
return OutputCSVFileRDD(self, path, dialect, overwrite, compress).collect()
def saveAsBinaryFile(self, path, fmt, overwrite=True):
return OutputBinaryFileRDD(self, path, fmt, overwrite).collect()
def saveAsTableFile(self, path, overwrite=True):
return OutputTableFileRDD(self, path, overwrite).collect()
def saveAsBeansdb(self, path, depth=0, overwrite=True, compress=True,
raw=False, valueWithMeta=False):
""" save (key, value) pair in beansdb format files
Args:
depth: choice = [0, 1, 2].
e.g. depth = 2 will write split N to 256 files:
'path/[0-F]/[0-F]/%03d.data' % N
MUST use depth == 0 to generate data for rivendb
raw: same as in DparkContext.beansdb
valueWithMeta: expect TRIPLE as input value
"""
assert depth <= 2, 'only support depth<=2 now'
if len(self) >= 256:
self = self.mergeSplit(len(self) // 256 + 1)
return OutputBeansdbRDD(self, path, depth, overwrite, compress,
raw, valueWithMeta).collect()
def saveAsTabular(self, path, field_names, **kw):
from dpark.tabular import OutputTabularRDD
return OutputTabularRDD(self, path, field_names, **kw).collect()
# Extra functions for (K,V) pairs RDD
def reduceByKeyToDriver(self, func):
def mergeMaps(m1, m2):
for k, v in six.iteritems(m2):
m1[k] = func(m1[k], v) if k in m1 else v
return m1
return self.map(lambda x_y2: {x_y2[0]: x_y2[1]}).reduce(mergeMaps)
def combineByKey(self, aggregator, splits=None, taskMemory=None, fixSkew=-1, rddconf=None):
if splits is None:
splits = min(self.ctx.defaultMinSplits, len(self))
if type(splits) is int:
_thresh = None
if fixSkew > 0 and splits > 1:
_step = 100. / splits
_offsets = [_step * i for i in range(1, splits)]
_percentiles = self.percentiles(
_offsets, sampleRate=fixSkew, func=lambda t: portable_hash(t[0])
)
if _percentiles:
_thresh = []
for p in _percentiles:
if math.isnan(p):
continue
p = int(math.ceil(p))
if not _thresh or p > _thresh[-1]:
_thresh.append(p)
if len(_thresh) + 1 < splits:
logger.warning('Highly skewed dataset detected!')
splits = len(_thresh) + 1
else:
_thresh = None
splits = HashPartitioner(splits, thresholds=_thresh)
return ShuffledRDD(self, aggregator, splits, taskMemory, rddconf=rddconf)
def reduceByKey(self, func, numSplits=None, taskMemory=None, fixSkew=-1, rddconf=None):
aggregator = Aggregator(lambda x: x, func, func)
return self.combineByKey(aggregator, numSplits, taskMemory, fixSkew=fixSkew, rddconf=rddconf)
def groupByKey(self, numSplits=None, taskMemory=None, fixSkew=-1, rddconf=None):
aggregator = GroupByAggregator()
return self.combineByKey(aggregator, numSplits, taskMemory, fixSkew=fixSkew,
rddconf=rddconf)
def topByKey(self, top_n, order_func=None,
reverse=False, num_splits=None, task_memory=None, fixSkew=-1):
""" Base on groupByKey, return the top_n values in each group.
The values in a key are ordered by a function object order_func.
The result values in a key is order in inc order, if you want
dec order, reverse is needed.
We implement the top n function of values in heap. To keep stable in the input values,
we trans the value in the records of rdd into a 4-tuple and sort them on heap.
The values with same return under order function will be sorted by their order added to heap,
only oldest value will be reserve.
After call of combineByKey, we call the map to unpack the value from 4-tuple.
:param top_n: required, limit of values to reserve in each group of a key
:param order_func: optional, order function map a value to a comparable value, default None, when
None, the value itself
:param reverse: optional, bool, when True, the value is sorted in dec order, else inc order
:param num_splits: same with groupByKey
:param task_memory: same with groupByKey
:return: rdd
"""
# To keep stable in heap, topByKey func need to generate the 4-tuple which is
# in the form of the
# (order_func(v), partition id, sequence id, v), in the end of the combineByKey
# return the v in the 4-th of the tuple
def get_tuple_list(s_id, it, rev):
return [(
i_k_v[1][0],
(order_func(i_k_v[1][1]) if order_func else i_k_v[1][1],
s_id if not rev else -s_id,
i_k_v[0] if not rev else -i_k_v[0],
i_k_v[1][1])
) for i_k_v in enumerate(it)]
aggregator = HeapAggregator(top_n,
order_reverse=reverse)
rdd = EnumeratePartitionsRDD(
self, lambda s_id, it: get_tuple_list(s_id, it, reverse)
)
return rdd.combineByKey(
aggregator, num_splits, task_memory, fixSkew=fixSkew
) \
.map(lambda x_ls: (x_ls[0], sorted(x_ls[1], reverse=reverse))) \
.map(lambda k_ls: (k_ls[0], [x[-1] for x in k_ls[1]]))
def partitionByKey(self, numSplits=None, taskMemory=None, rddconf=None):
return self.groupByKey(numSplits, taskMemory, rddconf=rddconf).flatMapValue(lambda x: x)
def update(self, other, replace_only=False, numSplits=None,
taskMemory=None, fixSkew=-1, rddconf=None):
rdd = self.mapValue(
lambda val: (val, 1) # bin('01') for old rdd
).union(
other.mapValue(
lambda val: (val, 2) # bin('10') for new rdd
)
).reduceByKey(
lambda x, y: (y[0] if y[1] > x[1] else x[0], x[1] | y[1]),
numSplits,
taskMemory,
fixSkew=fixSkew,
rddconf=rddconf
)
# rev:
# 1(01): old value
# 2(10): new added value
# 3(11): new updated value
if replace_only:
rdd = rdd.filter(
lambda key_val_rev: key_val_rev[1][1] != 2
)
return rdd.mapValue(
lambda val_rev: val_rev[0]
)
def innerJoin(self, smallRdd):
"""
This is functionally equivalent to `join`, but `innerJoin` assume `smallRdd` is a
small Data set, and `innerJoin` will broadcast the `smallRdd` to optimize running time.
>>> x = dpark_context.parallelize([("a", 1), ("b", 4)])
>>> y = dpark_context.parallelize([("a", 2), ("a", 3)])
>>> x.innerJoin(y).collect()
[('a', (1, 2)), ('a', (1, 3))]
"""
o = collections.defaultdict(list)
for (k, v) in smallRdd:
o[k].append(v)
o_b = self.ctx.broadcast(o)
def do_join(k_v):
(k, v) = k_v
for v1 in o_b.value[k]:
yield (k, (v, v1))
r = self.flatMap(do_join)
r.mem += (o_b.bytes * 10) >> 20 # memory used by broadcast obj
return r
def join(self, other, numSplits=None, taskMemory=None, fixSkew=-1, rddconf=None):
return self._join(other, (), numSplits, taskMemory, fixSkew=fixSkew, rddconf=rddconf)
def leftOuterJoin(self, other, numSplits=None, taskMemory=None, fixSkew=-1, rddconf=None):
return self._join(other, (1,), numSplits, taskMemory, fixSkew=fixSkew, rddconf=rddconf)
def rightOuterJoin(self, other, numSplits=None, taskMemory=None, fixSkew=-1, rddconf=None):
return self._join(other, (2,), numSplits, taskMemory, fixSkew=fixSkew, rddconf=rddconf)
def outerJoin(self, other, numSplits=None, taskMemory=None, fixSkew=-1, rddconf=None):
return self._join(other, (1, 2), numSplits, taskMemory, fixSkew=fixSkew, rddconf=rddconf)
def _join(self, other, keeps, numSplits=None, taskMemory=None, fixSkew=-1, rddconf=None):
def dispatch(k_seq):
(k, seq) = k_seq
vbuf, wbuf = seq
if not isinstance(wbuf, (list, tuple)):
wbuf = list(wbuf)
if not vbuf and 2 in keeps:
vbuf.append(None)
if not wbuf and 1 in keeps:
wbuf.append(None)
for vv in vbuf:
for ww in wbuf:
yield (k, (vv, ww))
return self.cogroup(other, numSplits, taskMemory, fixSkew=fixSkew, rddconf=rddconf) \
.flatMap(dispatch)
def collectAsMap(self):
d = {}
for v in self.ctx.runJob(self, lambda x: list(x)):
d.update(dict(v))
return d
def mapValue(self, f):
return MappedValuesRDD(self, f)
def flatMapValue(self, f):
return FlatMappedValuesRDD(self, f)
def groupWith(self, others, numSplits=None, taskMemory=None, fixSkew=-1, rddconf=None):
if isinstance(others, RDD):
others = [others]
_numSplits = numSplits
if _numSplits is None:
if self.partitioner is not None:
_numSplits = self.partitioner.numPartitions
else:
_numSplits = self.ctx.defaultParallelism
_thresh = None
if fixSkew > 0 and _numSplits > 1:
_step = 100. / _numSplits
_offsets = [_step * i for i in range(1, _numSplits)]
_percentiles = self.union(*others) \
.percentiles(
_offsets, sampleRate=fixSkew, func=lambda t: portable_hash(t[0])
)
if _percentiles:
_thresh = []
for p in _percentiles:
if math.isnan(p):
continue
p = int(math.ceil(p))
if not _thresh or p > _thresh[-1]:
_thresh.append(p)
if len(_thresh) + 1 < _numSplits:
logger.warning('Highly skewed dataset detected!')
_numSplits = len(_thresh) + 1
else:
_thresh = None
part = HashPartitioner(_numSplits, thresholds=_thresh)
rdd = CoGroupedRDD([self] + others, part, taskMemory, rddconf=rddconf)
return rdd
cogroup = groupWith
def lookup(self, key):
if self.partitioner:
index = self.partitioner.getPartition(key)
def process(it):
for k, v in it:
if k == key:
return v
result = list(self.ctx.runJob(self, process, [index], False))
return result[0] if result else None
else:
logger.warning("Too much time may be taken to lookup in a RDD without a partitioner!")
result = self.flatMap(lambda k_v: [k_v[1]] if k_v[0] == key else []).take(1)
return result[0] if result else None
def asTable(self, fields, name=''):
from dpark.table import TableRDD
return TableRDD(self, fields, name)
def batch(self, size):
def _batch(iterable):
sourceiter = iter(iterable)
while True:
s = list(itertools.islice(sourceiter, size))
if s:
yield s
else:
return
return self.glom().flatMap(_batch)
def adcount(self):
"approximate distinct counting"
r = self.map(lambda x: (1, x)).adcountByKey(1).collectAsMap()
return r and r[1] or 0
def adcountByKey(self, splits=None, taskMemory=None, fixSkew=-1):
try:
from pyhll import HyperLogLog
except ImportError:
from dpark.utils.hyperloglog import HyperLogLog
def create(v):
return HyperLogLog([v], 16)
def combine(s, v):
return s.add(v) or s
def merge(s1, s2):
return s1.update(s2) or s1
agg = Aggregator(create, combine, merge)
return self.combineByKey(agg, splits, taskMemory, fixSkew=fixSkew) \
.mapValue(len)
def percentiles(self, p, sampleRate=1.0, func=None):
def _(it):
from dpark.utils.tdigest import TDigest
digest = TDigest()
for k in it:
digest.add(k)
digest.compress()
yield digest
if sampleRate <= 0:
raise ValueError('Sample Rate should be positive.')
if sampleRate >= 1.0:
rdd = self
else:
rdd = self.sample(sampleRate)
if func:
rdd = rdd.map(func)
_digest = rdd.mapPartitions(_).reduce(lambda x, y: x + y)
_digest.compress()
return [_digest.quantile(pp / 100.) for pp in p]
def percentilesByKey(self, p, sampleRate=1.0, func=None,
numSplits=None, taskMemory=None, fixSkew=-1):
def _create(x):
from dpark.utils.tdigest import TDigest
digest = TDigest()
digest.add(x)
return digest
def _update(d, x):
d.add(x)
return d
def _merge(d1, d2):
d = d1 + d2
d.compress()
return d
def _(d):
return [d.quantile(pp / 100.) for pp in p]
if sampleRate <= 0:
raise ValueError('Sample Rate should be positive.')
if sampleRate >= 1.0:
rdd = self
else:
rdd = self.sample(sampleRate)
if func:
rdd = rdd.mapValue(func)
aggregator = Aggregator(_create, _update, _merge)
return rdd.combineByKey(aggregator, numSplits, taskMemory, fixSkew=fixSkew) \
.mapValue(_)
def with_cpus(self, cpus):
self.cpus = cpus
return self
def with_gpus(self, gpus):
self.gpus = gpus
return self
def with_mem(self, mem):
self.mem = mem
return self
def check_err_rate(self, total, err, end=False):
if not self.exc_info:
self.exc_info = sys.exc_info()
threshold = total * self.err_ratio
if not end:
fail = total > 100 and err > threshold * 10
else:
fail = err > threshold
if fail:
msg_tmpl = "too many error occured: %s/%s=%s, %s"
msg = msg_tmpl % (err, total, float(err) / total, self.exc_info[1])
raise (DparkUserFatalError, DparkUserFatalError(msg), self.exc_info[2])
class DerivedRDD(RDD):
def __init__(self, rdd):
RDD.__init__(self, rdd.ctx)
self.prev = rdd
self.mem = max(self.mem, rdd.mem)
self.cpus = rdd.cpus
self.gpus = rdd.gpus
self._dependencies = [OneToOneDependency(rdd)]
self._splits = self.prev.splits
self._preferred_locs = self.prev._preferred_locs
self.repr_name = '<%s %s>' % (self.__class__.__name__, rdd)
self.lineage += rdd.lineage
def _clear_dependencies(self):
RDD._clear_dependencies(self)
self.prev = None
def num_stream(self):
return self.prev.num_stream()
@property
def splits(self):
if self._checkpoint_rdd:
return self._checkpoint_rdd.splits
return self._splits
class MappedRDD(DerivedRDD):
def __init__(self, prev, func=lambda x: x):
DerivedRDD.__init__(self, prev)
self.func = func
@property
def params(self):
return func_info(self.func)
def compute(self, split):
if not self.allow_err:
return (self.func(v) for v in self.prev.iterator(split))
return self._compute_with_error(split)
def _compute_with_error(self, split):
total, err = 0, 0
for v in self.prev.iterator(split):
try:
total += 1
yield self.func(v)
except Exception as e:
logger.warning("ignored record %r: %s", v, e)
err += 1
self.check_err_rate(total, err, False)
self.check_err_rate(total, err, True)
@cached
def __getstate__(self):
d = RDD.__getstate__(self)
del d['func']
return d, dump_func(self.func)
def __setstate__(self, state):
self.__dict__, code = state
try:
self.func = load_func(code)
except Exception:
raise
class FlatMappedRDD(MappedRDD):
def compute(self, split):
if not self.allow_err:
return chain(self.func(v) for v in self.prev.iterator(split))
return self._compute_with_error(split)
def _compute_with_error(self, split):
total, err = 0, 0
for v in self.prev.iterator(split):
try:
total += 1
for k in self.func(v):
yield k
except Exception as e:
logger.warning("ignored record %r: %s", v, e)
err += 1
self.check_err_rate(total, err, False)
self.check_err_rate(total, err, True)
class FilteredRDD(MappedRDD):
def compute(self, split):
if not self.allow_err:
return (v for v in self.prev.iterator(split) if self.func(v))
return self._compute_with_error(split)
def _compute_with_error(self, split):
total, err = 0, 0
for v in self.prev.iterator(split):
try:
total += 1
if self.func(v):
yield v
except Exception as e:
logger.warning("ignored record %r: %s", v, e)
err += 1
self.check_err_rate(total, err, False)
self.check_err_rate(total, err, True)
class GlommedRDD(DerivedRDD):
def compute(self, split):
yield list(self.prev.iterator(split))
class MapPartitionsRDD(MappedRDD):
def compute(self, split):
return self.func(self.prev.iterator(split))
class EnumeratePartitionsRDD(MappedRDD):
def compute(self, split):
return self.func(split.index, self.prev.iterator(split))
class PipedRDD(DerivedRDD):