2
2
from itertools import chain , ifilter , imap
3
3
import operator
4
4
5
- import logging
6
-
7
5
from pyspark .serializers import NoOpSerializer ,\
8
6
BatchedSerializer , CloudPickleSerializer , pack_long
9
7
from pyspark .rdd import _JavaStackTrace
@@ -25,64 +23,86 @@ def count(self):
25
23
26
24
"""
27
25
#TODO make sure count implementation, thiis different from what pyspark does
28
- return self .mapPartitions (lambda i : [sum (1 for _ in i )]).map (lambda x : (None , 1 ))
26
+ return self ._mapPartitions (lambda i : [sum (1 for _ in i )]).map (lambda x : (None , 1 ))
29
27
30
28
def _sum (self ):
31
29
"""
32
30
"""
33
- return self .mapPartitions (lambda x : [sum (x )]).reduce (operator .add )
31
+ return self ._mapPartitions (lambda x : [sum (x )]).reduce (operator .add )
34
32
35
33
def print_ (self ):
36
34
"""
35
+ Since print is reserved name for python, we cannot make a print method function.
36
+ This function prints serialized data in RDD in DStream because Scala and Java cannot
37
+ deserialized pickled python object. Please use DStream.pyprint() instead to print result.
38
+
39
+ Call DStream.print().
37
40
"""
38
- # print is a reserved name of Python. We cannot give print to function name
41
+ #hack to call print function in DStream
39
42
getattr (self ._jdstream , "print" )()
40
43
41
44
def pyprint (self ):
42
45
"""
46
+ Print the first ten elements of each RDD generated in this DStream. This is an output
47
+ operator, so this DStream will be registered as an output stream and there materialized.
48
+
43
49
"""
44
50
self ._jdstream .pyprint ()
45
51
46
52
def filter (self , f ):
47
53
"""
54
+ Return DStream containing only the elements that satisfy predicate.
48
55
"""
49
56
def func (iterator ): return ifilter (f , iterator )
50
- return self .mapPartitions (func )
57
+ return self ._mapPartitions (func )
51
58
52
59
def flatMap (self , f , preservesPartitioning = False ):
53
60
"""
61
+ Pass each value in the key-value pair DStream through flatMap function
62
+ without changing the keys: this also retains the original RDD's partition.
54
63
"""
55
64
def func (s , iterator ): return chain .from_iterable (imap (f , iterator ))
56
- return self .mapPartitionsWithIndex (func , preservesPartitioning )
65
+ return self ._mapPartitionsWithIndex (func , preservesPartitioning )
57
66
58
- def map (self , f , preservesPartitioning = False ):
67
+ def map (self , f ):
59
68
"""
69
+ Return DStream by applying a function to each element of DStream.
60
70
"""
61
71
def func (iterator ): return imap (f , iterator )
62
- return self .mapPartitions (func )
63
- #return PipelinedDStream(self, func, preservesPartitioning)
72
+ return self ._mapPartitions (func )
64
73
65
- def mapPartitions (self , f ):
74
+ def _mapPartitions (self , f ):
66
75
"""
76
+ Return a new DStream by applying a function to each partition of this DStream.
67
77
"""
68
78
def func (s , iterator ): return f (iterator )
69
- return self .mapPartitionsWithIndex (func )
79
+ return self ._mapPartitionsWithIndex (func )
70
80
71
- def mapPartitionsWithIndex (self , f , preservesPartitioning = False ):
81
+ def _mapPartitionsWithIndex (self , f , preservesPartitioning = False ):
72
82
"""
73
-
83
+ Return a new DStream by applying a function to each partition of this DStream,
84
+ While tracking the index of the original partition.
74
85
"""
75
86
return PipelinedDStream (self , f , preservesPartitioning )
76
87
77
- def reduce (self , func , numPartitions = None ):
88
+
89
+ def reduceByKey (self , func , numPartitions = None ):
78
90
"""
91
+ Merge the value for each key using an associative reduce function.
92
+
93
+ This will also perform the merging locally on each mapper before
94
+ sending resuls to reducer, similarly to a "combiner" in MapReduce.
79
95
96
+ Output will be hash-partitioned with C{numPartitions} partitions, or
97
+ the default parallelism level if C{numPartitions} is not specified.
80
98
"""
81
99
return self .combineByKey (lambda x :x , func , func , numPartitions )
82
100
83
101
def combineByKey (self , createCombiner , mergeValue , mergeCombiners ,
84
102
numPartitions = None ):
85
103
"""
104
+ Count the number of elements for each key, and return the result to the
105
+ master as a dictionary
86
106
"""
87
107
if numPartitions is None :
88
108
numPartitions = self ._defaultReducePartitions ()
@@ -148,42 +168,27 @@ def add_shuffle_key(split, iterator):
148
168
dstream ._partitionFunc = partitionFunc
149
169
return dstream
150
170
151
- def mapPartitionsWithIndex (self , f , preservesPartitioning = False ):
152
- """
153
-
154
- """
155
- return PipelinedDStream (self , f , preservesPartitioning )
156
-
157
171
def _defaultReducePartitions (self ):
158
172
"""
173
+ Returns the default number of partitions to use during reduce tasks (e.g., groupBy).
174
+ If spark.default.parallelism is set, then we'll use the value from SparkContext
175
+ defaultParallelism, otherwise we'll use the number of partitions in this RDD.
159
176
177
+ This mirrors the behavior of the Scala Partitioner#defaultPartitioner, intended to reduce
178
+ the likelihood of OOMs. Once PySpark adopts Partitioner-based APIs, this behavior will
179
+ be inherent.
160
180
"""
161
- # hard code to avoid the error
162
181
if self .ctx ._conf .contains ("spark.default.parallelism" ):
163
182
return self .ctx .defaultParallelism
164
183
else :
165
184
return self .getNumPartitions ()
166
185
167
- return self ._jdstream .partitions ().size ()
168
-
169
- def _defaultReducePartitions (self ):
186
+ def getNumPartitions (self ):
170
187
"""
171
-
188
+ Return the number of partitions in RDD
172
189
"""
173
- # hard code to avoid the error
174
- if self .ctx ._conf .contains ("spark.default.parallelism" ):
175
- return self .ctx .defaultParallelism
176
- else :
177
- return self .getNumPartitions ()
178
-
179
- def getNumPartitions (self ):
180
- """
181
- Returns the number of partitions in RDD
182
- >>> rdd = sc.parallelize([1, 2, 3, 4], 2)
183
- >>> rdd.getNumPartitions()
184
- 2
185
- """
186
- return self ._jdstream .partitions ().size ()
190
+ # TODO: remove hardcoding. RDD has NumPartitions but DStream does not have.
191
+ return 2
187
192
188
193
189
194
class PipelinedDStream (DStream ):
0 commit comments