-
Notifications
You must be signed in to change notification settings - Fork 44
/
rdd.py
2215 lines (1678 loc) · 65.3 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
"""Provides a Python implementation of RDDs."""
from __future__ import (division, absolute_import, print_function,
unicode_literals)
import math
from builtins import range, zip
from collections import defaultdict
import copy
import functools
import io
import itertools
import logging
from operator import itemgetter
import os
import pickle
import random
import subprocess
import sys
try:
import numpy
except ImportError:
numpy = None
from . import fileio
from .utils import portable_hash
from .exceptions import FileAlreadyExistsException, ContextIsLockedException
from .samplers import (BernoulliSampler, PoissonSampler,
BernoulliSamplerPerKey, PoissonSamplerPerKey)
from .stat_counter import StatCounter
maxint = sys.maxint if hasattr(sys, 'maxint') else sys.maxsize # pylint: disable=no-member
log = logging.getLogger(__name__)
def _hash(v):
return portable_hash(v) & 0xffffffff
class RDD(object):
"""RDD
In Spark's original form, RDDs are Resilient, Distributed Datasets.
This class reimplements the same interface with the goal of being
fast on small data at the cost of being resilient and distributed.
:param list partitions:
A list of instances of :class:`Partition`.
:param Context ctx:
An instance of the applicable :class:`Context`.
"""
def __init__(self, partitions, ctx):
if ctx.locked:
raise ContextIsLockedException
self._p = list(partitions)
self.context = ctx
self._name = None
self._rdd_id = ctx.newRddId()
def __getstate__(self):
r = {k: v if k not in ('_p',) else None
for k, v in self.__dict__.items()}
return r
def compute(self, split, task_context):
"""interface to extend behavior for specific cases
:param Partition split: a partition
"""
return split.x()
def partitions(self):
return self._p
"""
Public API
----------
"""
def aggregate(self, zeroValue, seqOp, combOp):
"""aggregate
[distributed]
:param zeroValue:
The initial value to an aggregation, for example ``0`` or ``0.0``
for aggregating `int` s and `float` s, but any Python object is
possible.
:param seqOp:
A reference to a function that combines the current state with a
new value. In the first iteration, the current state is zeroValue.
:param combOp:
A reference to a function that combines outputs of seqOp.
In the first iteration, the current state is zeroValue.
:returns:
Output of ``combOp`` operations.
Example:
>>> from pysparkling import Context
>>> seqOp = (lambda x, y: (x[0] + y, x[1] + 1))
>>> combOp = (lambda x, y: (x[0] + y[0], x[1] + y[1]))
>>> Context().parallelize(
... [1, 2, 3, 4], 2
... ).aggregate((0, 0), seqOp, combOp)
(10, 4)
"""
return self.context.runJob(
self,
lambda tc, i: functools.reduce(
seqOp, i, copy.deepcopy(zeroValue)
),
resultHandler=lambda l: functools.reduce(
combOp, l, copy.deepcopy(zeroValue)
),
)
def treeAggregate(self, zeroValue, seqOp, combOp, depth=2):
"""same internal behaviour as :func:`~pysparkling.RDD.aggregate()`
:param depth: Not used.
"""
return self.aggregate(zeroValue, seqOp, combOp)
def aggregateByKey(self, zeroValue, seqFunc, combFunc, numPartitions=None):
"""aggregate by key
:param zeroValue:
The initial value to an aggregation, for example ``0`` or ``0.0``
for aggregating `int` s and `float` s, but any Python object is
possible.
:param seqFunc:
A reference to a function that combines the current state with a
new value. In the first iteration, the current state is zeroValue.
:param combFunc:
A reference to a function that combines outputs of seqFunc.
In the first iteration, the current state is zeroValue.
:param int numPartitions: Not used.
:returns: An RDD with the output of ``combOp`` operations.
:rtype: RDD
Example:
>>> from pysparkling import Context
>>> seqOp = (lambda x, y: x + y)
>>> combOp = (lambda x, y: x + y)
>>> r = Context().parallelize(
... [('a', 1), ('b', 2), ('a', 3), ('c', 4)]
... ).aggregateByKey(0, seqOp, combOp).collectAsMap()
>>> (r['a'], r['b'])
(4, 2)
"""
def seqFuncByKey(tc, i):
r = defaultdict(lambda: copy.deepcopy(zeroValue))
for k, v in i:
r[k] = seqFunc(r[k], v)
return r
def combFuncByKey(l):
r = defaultdict(lambda: copy.deepcopy(zeroValue))
for p in l:
for k, v in p.items():
r[k] = combFunc(r[k], v)
return r
local_result = self.context.runJob(self, seqFuncByKey,
resultHandler=combFuncByKey)
return self.context.parallelize(local_result.items())
def cache(self):
"""Once a partition is computed, cache the result.
Alias for :func:`~pysparkling.RDD.persist`.
Example:
>>> import pysparkling
>>>
>>> n_exec = 0
>>>
>>> def _map(e):
... global n_exec
... n_exec += 1
... return e*e
>>>
>>> sc = pysparkling.Context()
>>> my_rdd = sc.parallelize([1, 2, 3, 4], 2).map(_map).cache()
>>>
>>> # no exec until here
>>> n_exec
0
>>> # to get first element, compute the first partition
>>> my_rdd.first()
1
>>> n_exec
2
>>> # now compute the rest
>>> my_rdd.collect()
[1, 4, 9, 16]
>>> n_exec
4
>>> # now _map() was executed on all partitions and should
>>> # not be executed again
>>> my_rdd.collect()
[1, 4, 9, 16]
>>> n_exec
4
"""
return self.persist()
def glom(self):
"""coalesce into a list elements of a partition
:rtype: RDD
>>> from pysparkling import Context
>>> rdd = Context().parallelize([1, 2, 3, 4], 2)
>>> sorted(rdd.glom().collect())
[[1, 2], [3, 4]]
"""
return self.mapPartitions(lambda items: [list(items)])
def cartesian(self, other):
"""cartesian product of this RDD with ``other``
:param RDD other: Another RDD.
:rtype: RDD
.. note::
This is currently implemented as a local operation requiring
all data to be pulled on one machine.
Example:
>>> from pysparkling import Context
>>> rdd = Context().parallelize([1, 2])
>>> sorted(rdd.cartesian(rdd).collect())
[(1, 1), (1, 2), (2, 1), (2, 2)]
"""
v1 = self.toLocalIterator()
v2 = other.collect()
return self.context.parallelize([(a, b) for a in v1 for b in v2])
def coalesce(self, numPartitions, shuffle=False):
"""coalesce
:param int numPartitions: Number of partitions in the resulting RDD.
:param shuffle: (optional) Not used.
:rtype: RDD
.. note::
This is currently implemented as a local operation requiring
all data to be pulled on one machine.
Example:
>>> from pysparkling import Context
>>> Context().parallelize([1, 2, 3], 2).coalesce(1).getNumPartitions()
1
>>> from pysparkling import Context
>>> Context().parallelize([1, 2, 3], 2).coalesce(4).getNumPartitions()
2
>>> from pysparkling import Context
>>> rdd = Context().parallelize([1, 2, 3, 4, 5, 6, 7, 8], 5)
>>> rdd.foreachPartition(lambda x: print(list(x)))
[1]
[2, 3]
[4]
[5, 6]
[7, 8]
>>> rdd.coalesce(4).foreachPartition(lambda x: print(list(x)))
[1, 2, 3]
[4]
[5, 6]
[7, 8]
>>> rdd.coalesce(4).coalesce(3).foreachPartition(lambda x: print(list(x)))
[1, 2, 3, 4]
[5, 6]
[7, 8]
>>> rdd.coalesce(3).foreachPartition(lambda x: print(list(x)))
[1, 2, 3]
[4, 5, 6]
[7, 8]
"""
if shuffle:
return self.context.parallelize(self.toLocalIterator(), numPartitions)
current_num_partitions = self.getNumPartitions()
new_num_partitions = min(numPartitions, current_num_partitions)
# Group partitions that will be coalesced together
# Note: as new_num_partitions may not divide current_num_partitions
# some groups are bigger (contains more partitions) than others
small_group_size = current_num_partitions // new_num_partitions
big_group_size = small_group_size + 1
number_of_big_groups = current_num_partitions % new_num_partitions
number_of_small_groups = new_num_partitions - number_of_big_groups
partition_mapping = ([p for p in range(number_of_big_groups)
for _ in range(big_group_size)] +
[p for p in range(number_of_big_groups,
number_of_big_groups + number_of_small_groups)
for _ in range(small_group_size)])
new_partitions = {i: [] for i in range(new_num_partitions)}
def partitioned():
def move_partition_content(partition_index, partition):
new_partitions[partition_mapping[partition_index]] += partition
return []
# trigger an evaluation with count
self.mapPartitionsWithIndex(move_partition_content).count()
for p in list(new_partitions.values()):
yield p
# noinspection PyProtectedMember
return self.context._parallelize_partitions(partitioned())
def cogroup(self, other, numPartitions=None):
"""Groups keys from both RDDs together. Values are nested iterators.
:param RDD other: The other RDD.
:param int numPartitions: Number of partitions in the resulting RDD.
:rtype: RDD
Example:
>>> from pysparkling import Context
>>> c = Context()
>>> a = c.parallelize([('house', 1), ('tree', 2)])
>>> b = c.parallelize([('house', 3)])
>>>
>>> [(k, sorted(list([list(vv) for vv in v])))
... for k, v in sorted(a.cogroup(b).collect())
... ]
[('house', [[1], [3]]), ('tree', [[], [2]])]
"""
d_self = defaultdict(list, self.groupByKey().collect())
d_other = defaultdict(list, other.groupByKey().collect())
return self.context.parallelize([
(k, [list(d_self[k]), list(d_other[k])])
for k in set(d_self.keys()) | set(d_other.keys())
])
def collect(self):
"""returns the entire dataset as a list
:rtype: list
Example:
>>> from pysparkling import Context
>>> Context().parallelize([1, 2, 3]).collect()
[1, 2, 3]
"""
return self.context.runJob(
self,
unit_map,
resultHandler=unit_collect,
)
def collectAsMap(self):
"""returns a dictionary for a pair dataset
:rtype: dict
Example:
>>> from pysparkling import Context
>>> d = Context().parallelize([('a', 1), ('b', 2)]).collectAsMap()
>>> (d['a'], d['b'])
(1, 2)
"""
return dict(self.collect())
def count(self):
"""number of entries in this dataset
:rtype: int
Example:
>>> from pysparkling import Context
>>> Context().parallelize([1, 2, 3], 2).count()
3
"""
return self.context.runJob(self, lambda tc, i: sum(1 for _ in i),
resultHandler=sum)
def countApprox(self):
"""same as :func:`~pysparkling.RDD.count()`
:rtype: int
"""
return self.count()
def countApproxDistinct(self):
"""return the number of distinct values
:rtype: int
.. note::
The operation is currently implemented as a local and exact operation.
"""
return len(set(self.toLocalIterator()))
def countByKey(self):
"""returns a `dict` containing the count for every key
:rtype: dict
Example:
>>> from pysparkling import Context
>>> Context().parallelize(
... [('a', 1), ('b', 2), ('b', 2)]
... ).countByKey()['b']
2
"""
return self.map(lambda r: r[0]).countByValue()
def countByValue(self):
"""returns a `dict` containing the count for every value
:rtype: dict
Example:
>>> from pysparkling import Context
>>> Context().parallelize([1, 2, 2, 4, 1]).countByValue()[2]
2
"""
def map_func(tc, x):
r = defaultdict(int)
for v in x:
r[v] += 1
return r
return self.context.runJob(self, map_func,
resultHandler=sum_counts_by_keys)
def distinct(self, numPartitions=None):
"""returns only distinct elements
:param int numPartitions: Number of partitions in the resulting RDD.
:rtype: RDD
Example:
>>> from pysparkling import Context
>>> Context().parallelize([1, 2, 2, 4, 1]).distinct().count()
3
"""
if numPartitions is None:
numPartitions = self.getNumPartitions()
return self.context.parallelize(list(set(self.toLocalIterator())),
numPartitions)
def filter(self, f):
"""filter elements
:param f: a function that decides whether to keep an element
:rtype: RDD
Example:
>>> from pysparkling import Context
>>> Context().parallelize(
... [1, 2, 2, 4, 1, 3, 5, 9], 3,
... ).filter(lambda x: x % 2 == 0).collect()
[2, 2, 4]
"""
return MapPartitionsRDD(
self,
lambda tc, i, x: (xx for xx in x if f(xx)),
preservesPartitioning=True,
)
def first(self):
"""returns the first element in the dataset
Example:
>>> from pysparkling import Context
>>> Context().parallelize([1, 2, 2, 4, 1, 3, 5, 9], 3).first()
1
Works also with empty partitions:
>>> from pysparkling import Context
>>> Context().parallelize([1, 2], 20).first()
1
"""
return self.context.runJob(
self,
lambda tc, iterable: iterable,
allowLocal=True,
resultHandler=lambda l: next(itertools.chain.from_iterable(l)),
)
def flatMap(self, f, preservesPartitioning=True):
"""map followed by flatten
:param f: The map function.
:param preservesPartitioning: (optional) Preserve the partitioning of
the original RDD. Default True.
:rtype: RDD
Example:
>>> from pysparkling import Context
>>> Context().parallelize(['hello', 'world']).flatMap(
... lambda x: [ord(ch) for ch in x]
... ).collect()
[104, 101, 108, 108, 111, 119, 111, 114, 108, 100]
"""
return MapPartitionsRDD(
self,
lambda tc, i, x: (e for xx in x for e in f(xx)),
preservesPartitioning=preservesPartitioning,
)
def flatMapValues(self, f):
"""map operation on values in a (key, value) pair followed by a flatten
:param f: The map function.
:rtype: RDD
Example:
>>> from pysparkling import Context
>>> Context().parallelize([(1, 'hi'), (2, 'world')]).flatMapValues(
... lambda x: [ord(ch) for ch in x]
... ).collect()
[(1, 104), (1, 105), (2, 119), (2, 111), (2, 114), (2, 108), (2, 100)]
"""
return MapPartitionsRDD(
self,
lambda tc, i, x: ((xx[0], e) for xx in x for e in f(xx[1])),
preservesPartitioning=True,
)
def fold(self, zeroValue, op):
"""fold
:param zeroValue: The inital value, for example ``0`` or ``0.0``.
:param op: The reduce operation.
:returns: The folded (or aggregated) value.
Example:
>>> from pysparkling import Context
>>> my_rdd = Context().parallelize([4, 7, 2])
>>> my_rdd.fold(0, lambda a, b: a+b)
13
"""
return self.aggregate(zeroValue, op, op)
def foldByKey(self, zeroValue, op):
"""Fold (or aggregate) value by key.
:param zeroValue: The inital value, for example ``0`` or ``0.0``.
:param op: The reduce operation.
:rtype: RDD
Example:
>>> from pysparkling import Context
>>> my_rdd = Context().parallelize([('a', 4), ('b', 7), ('a', 2)])
>>> my_rdd.foldByKey(0, lambda a, b: a+b).collectAsMap()['a']
6
"""
return self.aggregateByKey(zeroValue, op, op)
def foreach(self, f):
"""applies ``f`` to every element
It does not return a new RDD like :func:`~pysparkling.RDD.map`.
:param f: Apply a function to every element.
:rtype: None
Example:
>>> from pysparkling import Context
>>> my_rdd = Context().parallelize([1, 2, 3])
>>> a = []
>>> my_rdd.foreach(lambda x: a.append(x))
>>> len(a)
3
"""
self.context.runJob(self, lambda tc, x: [f(xx) for xx in x],
resultHandler=None)
def foreachPartition(self, f):
"""applies ``f`` to every partition
It does not return a new RDD like
:func:`~pysparkling.RDD.mapPartitions`.
:param f: Apply a function to every partition.
:rtype: None
"""
self.context.runJob(self, lambda tc, x: f(x),
resultHandler=None)
def fullOuterJoin(self, other, numPartitions=None):
"""returns the full outer join of two RDDs
The output contains all keys from both input RDDs, with missing
keys replaced with `None`.
:param RDD other: The RDD to join to this one.
:param int numPartitions: Number of partitions in the resulting RDD.
:rtype: RDD
.. note::
Creating the new RDD is currently implemented as a local operation.
Example:
>>> from pysparkling import Context
>>> sc = Context()
>>> rdd1 = sc.parallelize([('a', 0), ('b', 1)])
>>> rdd2 = sc.parallelize([('b', 2), ('c', 3)])
>>> sorted(
... rdd1.fullOuterJoin(rdd2).collect()
... )
[('a', (0, None)), ('b', (1, 2)), ('c', (None, 3))]
"""
grouped = self.cogroup(other, numPartitions)
return grouped.flatMap(lambda kv: [
(kv[0], (v_self, v_other))
for v_self in (kv[1][0] if kv[1][0] else [None])
for v_other in (kv[1][1] if kv[1][1] else [None])
])
def getNumPartitions(self):
"""returns the number of partitions
:rtype: int
"""
return len(self.partitions())
def getPartitions(self):
"""returns the partitions of this RDD
:rtype: list
"""
return self.partitions()
def groupBy(self, f, numPartitions=None):
"""group by f
:param f: Function returning a key given an element of the dataset.
:param int numPartitions: Number of partitions in the resulting RDD.
:rtype: RDD
.. note::
Creating the new RDD is currently implemented as a local operation.
Example:
>>> from pysparkling import Context
>>> my_rdd = Context().parallelize([4, 7, 2])
>>> my_rdd.groupBy(lambda x: x % 2).mapValues(sorted).collect()
[(0, [2, 4]), (1, [7])]
"""
return self.keyBy(f).groupByKey(numPartitions)
def groupByKey(self, numPartitions=None):
"""group by key
:param int numPartitions: Number of partitions in the resulting RDD.
:rtype: RDD
.. note::
Creating the new RDD is currently implemented as a local operation.
"""
if numPartitions is None:
numPartitions = self.getNumPartitions()
r = defaultdict(list)
for key, value in self.toLocalIterator():
r[key].append(value)
return self.context.parallelize(r.items(), numPartitions)
def histogram(self, buckets):
"""histogram
:param buckets:
A list of bucket boundaries or an int for the number of buckets.
:returns:
A tuple (bucket_boundaries, histogram_values) where
bucket_boundaries is a list of length n+1 boundaries and
histogram_values is a list of length n with the values of each
bucket.
Example:
>>> from pysparkling import Context
>>> my_rdd = Context().parallelize([0, 4, 7, 4, 10])
>>> b, h = my_rdd.histogram(10)
>>> h
[1, 0, 0, 0, 2, 0, 0, 1, 0, 0, 1]
"""
if isinstance(buckets, int):
num_buckets = buckets
stats = self.stats()
min_v = stats.min()
max_v = stats.max()
buckets = [min_v + float(i) * (max_v - min_v) / num_buckets
for i in range(num_buckets + 1)]
h = [0 for _ in buckets]
for x in self.toLocalIterator():
for i, b in enumerate(zip(buckets[:-1], buckets[1:])):
if b[0] <= x < b[1]:
h[i] += 1
# make the last bin inclusive on the right
if x == buckets[-1]:
h[-1] += 1
return (buckets, h)
def id(self):
"""the id of this RDD"""
return self._rdd_id
def intersection(self, other):
"""intersection of this and other RDD
:param RDD other: The other dataset to do the intersection with.
:rtype: RDD
.. note::
Creating the new RDD is currently implemented as a local operation.
Example:
>>> from pysparkling import Context
>>> rdd1 = Context().parallelize([0, 4, 7, 4, 10])
>>> rdd2 = Context().parallelize([3, 4, 7, 4, 5])
>>> rdd1.intersection(rdd2).collect()
[4, 7]
"""
return self.context.parallelize(
list(set(self.toLocalIterator()) & set(other.toLocalIterator()))
)
def isCheckpointed(self):
return False
def join(self, other, numPartitions=None):
"""join
:param RDD other: The other RDD.
:param int numPartitions: Number of partitions in the resulting RDD.
:rtype: RDD
.. note::
Creating the new RDD is currently implemented as a local operation.
Example:
>>> from pysparkling import Context
>>> rdd1 = Context().parallelize([(0, 1), (1, 1)])
>>> rdd2 = Context().parallelize([(2, 1), (1, 3)])
>>> rdd1.join(rdd2).collect()
[(1, (1, 3))]
"""
if numPartitions is None:
numPartitions = self.getNumPartitions()
d1 = dict(self.collect())
d2 = dict(other.collect())
keys = set(d1.keys()) & set(d2.keys())
return self.context.parallelize((
(k, (d1[k], d2[k]))
for k in keys
), numPartitions)
def keyBy(self, f):
"""key by f
:param f: Function that returns a key from a dataset element.
:rtype: RDD
Example:
>>> from pysparkling import Context
>>> rdd = Context().parallelize([0, 4, 7, 4, 10])
>>> rdd.keyBy(lambda x: x % 2).collect()
[(0, 0), (0, 4), (1, 7), (0, 4), (0, 10)]
"""
return self.map(lambda e: (f(e), e))
def keys(self):
"""keys of a pair dataset
:rtype: RDD
Example:
>>> from pysparkling import Context
>>> Context().parallelize([(0, 1), (1, 1)]).keys().collect()
[0, 1]
"""
return self.map(lambda e: e[0])
def leftOuterJoin(self, other, numPartitions=None):
"""left outer join
:param RDD other: The other RDD.
:param int numPartitions: Number of partitions in the resulting RDD.
:rtype: RDD
.. note::
Creating the new RDD is currently implemented as a local operation.
Example:
>>> from pysparkling import Context
>>> rdd1 = Context().parallelize([(0, 1), (1, 1)])
>>> rdd2 = Context().parallelize([(2, 1), (1, 3)])
>>> rdd1.leftOuterJoin(rdd2).collect()
[(0, (1, None)), (1, (1, 3))]
"""
d_other = other.groupByKey().collectAsMap()
return self.groupByKey().flatMap(lambda kv: [
(kv[0], (v_self, v_other))
for v_self in kv[1]
for v_other in (d_other[kv[0]] if kv[0] in d_other else [None])
])
def _leftSemiJoin(self, other):
"""left semi join
This function is not part of the official Spark API hence its leading "_"
:param RDD other: The other RDD.
:rtype: RDD
.. note::
Creating the new RDD is currently implemented as a local operation.
Example:
>>> from pysparkling import Context
>>> rdd1 = Context().parallelize([(0, 1), (1, 1)])
>>> rdd2 = Context().parallelize([(2, 1), (1, 3)])
>>> rdd1._leftSemiJoin(rdd2).collect()
[(1, (1, ()))]
"""
d_other = other.groupByKey().collectAsMap()
return self.groupByKey().flatMap(lambda kv: [
(kv[0], (v_self, ()))
for v_self in kv[1]
if kv[0] in d_other
])
def _leftAntiJoin(self, other):
"""left anti join
This function is not part of the official Spark API hence its leading "_"
:param RDD other: The other RDD.
:param int numPartitions: Number of partitions in the resulting RDD.
:rtype: RDD
.. note::
Creating the new RDD is currently implemented as a local operation.
Example:
>>> from pysparkling import Context
>>> rdd1 = Context().parallelize([(0, 1), (1, 1)])
>>> rdd2 = Context().parallelize([(2, 1), (1, 3)])
>>> rdd1._leftAntiJoin(rdd2).collect()
[(0, (1, None))]
"""
d_other = other.groupByKey().collectAsMap()
return self.groupByKey().flatMap(lambda kv: [
(kv[0], (v_self, None))
for v_self in kv[1]
if kv[0] not in d_other
])
def lookup(self, key):
"""Return all the (key, value) pairs where the given key matches.
:param key: The key to lookup.
:rtype: list
Example:
>>> from pysparkling import Context
>>> Context().parallelize([(0, 1), (1, 1), (1, 3)]).lookup(1)
[1, 3]
"""
return self.filter(lambda x: x[0] == key).values().collect()
def map(self, f):
"""map
:param f: map function for elements
:rtype: RDD
Example:
>>> from pysparkling import Context
>>> Context().parallelize([1, 2, 3]).map(lambda x: x+1).collect()
[2, 3, 4]
"""
return MapPartitionsRDD(
self,
MapF(f),
preservesPartitioning=True,
).setName('{}:{}'.format(self.name(), f))
def mapPartitions(self, f, preservesPartitioning=False):
"""map partitions
:param f: map function for partitions
:rtype: RDD
Example:
>>> from pysparkling import Context
>>> rdd = Context().parallelize([1, 2, 3, 4], 2)
>>> def f(iterator):
... yield sum(iterator)
>>> rdd.mapPartitions(f).collect()
[3, 7]
"""
return MapPartitionsRDD(
self,