@@ -22,25 +22,23 @@ def count(self):
22
22
"""
23
23
24
24
"""
25
- pass
26
- #TODO: make sure count implementation, thiis different from what pyspark does
27
- #return self._mapPartitions(lambda i: [sum(1 for _ in i)]).map(lambda x: (None, 1))
25
+ # TODO: make sure count implementation, this different from what pyspark does
26
+ return self ._mapPartitions (lambda i : [sum (1 for _ in i )])._sum ()
28
27
29
28
def _sum (self ):
30
29
"""
31
30
"""
32
- pass
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
"""
37
- Since print is reserved name for python, we cannot make a print method function.
35
+ Since print is reserved name for python, we cannot define a print method function.
38
36
This function prints serialized data in RDD in DStream because Scala and Java cannot
39
- deserialized pickled python object. Please use DStream.pyprint() instead to print result .
37
+ deserialized pickled python object. Please use DStream.pyprint() instead to print results .
40
38
41
39
Call DStream.print().
42
40
"""
43
- #hack to call print function in DStream
41
+ # a hack to call print function in DStream
44
42
getattr (self ._jdstream , "print" )()
45
43
46
44
def filter (self , f ):
@@ -79,17 +77,23 @@ def _mapPartitionsWithIndex(self, f, preservesPartitioning=False):
79
77
"""
80
78
return PipelinedDStream (self , f , preservesPartitioning )
81
79
80
+ def reduce (self , func ):
81
+ """
82
+
83
+ """
84
+ return self .map (lambda x : (None , x )).reduceByKey (func , 1 ).map (lambda x : x [1 ])
85
+
82
86
def reduceByKey (self , func , numPartitions = None ):
83
87
"""
84
88
Merge the value for each key using an associative reduce function.
85
89
86
90
This will also perform the merging locally on each mapper before
87
- sending resuls to reducer, similarly to a "combiner" in MapReduce.
91
+ sending results to reducer, similarly to a "combiner" in MapReduce.
88
92
89
93
Output will be hash-partitioned with C{numPartitions} partitions, or
90
94
the default parallelism level if C{numPartitions} is not specified.
91
95
"""
92
- return self .combineByKey (lambda x :x , func , func , numPartitions )
96
+ return self .combineByKey (lambda x : x , func , func , numPartitions )
93
97
94
98
def combineByKey (self , createCombiner , mergeValue , mergeCombiners ,
95
99
numPartitions = None ):
@@ -99,6 +103,7 @@ def combineByKey(self, createCombiner, mergeValue, mergeCombiners,
99
103
"""
100
104
if numPartitions is None :
101
105
numPartitions = self ._defaultReducePartitions ()
106
+
102
107
def combineLocally (iterator ):
103
108
combiners = {}
104
109
for x in iterator :
@@ -116,6 +121,7 @@ def combineLocally(iterator):
116
121
return combiners .iteritems ()
117
122
locally_combined = self ._mapPartitions (combineLocally )
118
123
shuffled = locally_combined .partitionBy (numPartitions )
124
+
119
125
def _mergeCombiners (iterator ):
120
126
combiners = {}
121
127
for (k , v ) in iterator :
@@ -124,6 +130,7 @@ def _mergeCombiners(iterator):
124
130
else :
125
131
combiners [k ] = mergeCombiners (combiners [k ], v )
126
132
return combiners .iteritems ()
133
+
127
134
return shuffled ._mapPartitions (_mergeCombiners )
128
135
129
136
def partitionBy (self , numPartitions , partitionFunc = None ):
0 commit comments