22from  itertools  import  chain , ifilter , imap 
33import  operator 
44
5- import  logging 
6- 
75from  pyspark .serializers  import  NoOpSerializer ,\
86    BatchedSerializer , CloudPickleSerializer , pack_long 
97from  pyspark .rdd  import  _JavaStackTrace 
@@ -25,64 +23,86 @@ def count(self):
2523
2624        """ 
2725        #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 ))
2927
3028    def  _sum (self ):
3129        """ 
3230        """ 
33-         return  self .mapPartitions (lambda  x : [sum (x )]).reduce (operator .add )
31+         return  self ._mapPartitions (lambda  x : [sum (x )]).reduce (operator .add )
3432
3533    def  print_ (self ):
3634        """ 
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(). 
3740        """ 
38-         # print is a reserved name of Python. We cannot give  print to  function name  
41+         #hack to call  print function in DStream  
3942        getattr (self ._jdstream , "print" )()
4043
4144    def  pyprint (self ):
4245        """ 
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+ 
4349        """ 
4450        self ._jdstream .pyprint ()
4551
4652    def  filter (self , f ):
4753        """ 
54+         Return DStream containing only the elements that satisfy predicate. 
4855        """ 
4956        def  func (iterator ): return  ifilter (f , iterator )
50-         return  self .mapPartitions (func )
57+         return  self ._mapPartitions (func )
5158
5259    def  flatMap (self , f , preservesPartitioning = False ):
5360        """ 
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. 
5463        """ 
5564        def  func (s , iterator ): return  chain .from_iterable (imap (f , iterator ))
56-         return  self .mapPartitionsWithIndex (func , preservesPartitioning )
65+         return  self ._mapPartitionsWithIndex (func , preservesPartitioning )
5766
58-     def  map (self , f ,  preservesPartitioning = False ):
67+     def  map (self , f ):
5968        """ 
69+         Return DStream by applying a function to each element of DStream. 
6070        """ 
6171        def  func (iterator ): return  imap (f , iterator )
62-         return  self .mapPartitions (func )
63-         #return PipelinedDStream(self, func, preservesPartitioning) 
72+         return  self ._mapPartitions (func )
6473
65-     def  mapPartitions (self , f ):
74+     def  _mapPartitions (self , f ):
6675        """ 
76+         Return a new DStream by applying a function to each partition of this DStream. 
6777        """ 
6878        def  func (s , iterator ): return  f (iterator )
69-         return  self .mapPartitionsWithIndex (func )
79+         return  self ._mapPartitionsWithIndex (func )
7080
71-     def  mapPartitionsWithIndex (self , f , preservesPartitioning = False ):
81+     def  _mapPartitionsWithIndex (self , f , preservesPartitioning = False ):
7282        """ 
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. 
7485        """ 
7586        return  PipelinedDStream (self , f , preservesPartitioning )
7687
77-     def  reduce (self , func , numPartitions = None ):
88+ 
89+     def  reduceByKey (self , func , numPartitions = None ):
7890        """ 
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. 
7995
96+         Output will be hash-partitioned with C{numPartitions} partitions, or 
97+         the default parallelism level if C{numPartitions} is not specified. 
8098        """ 
8199        return  self .combineByKey (lambda  x :x , func , func , numPartitions )
82100
83101    def  combineByKey (self , createCombiner , mergeValue , mergeCombiners ,
84102                      numPartitions  =  None ):
85103        """ 
104+         Count the number of elements for each key, and return the result to the 
105+         master as a dictionary 
86106        """ 
87107        if  numPartitions  is  None :
88108            numPartitions  =  self ._defaultReducePartitions ()
@@ -148,42 +168,27 @@ def add_shuffle_key(split, iterator):
148168        dstream ._partitionFunc  =  partitionFunc 
149169        return  dstream 
150170
151-     def  mapPartitionsWithIndex (self , f , preservesPartitioning = False ):
152-         """ 
153- 
154-         """ 
155-         return  PipelinedDStream (self , f , preservesPartitioning )
156- 
157171    def  _defaultReducePartitions (self ):
158172        """ 
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. 
159176
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. 
160180        """ 
161-         # hard code to avoid the error 
162181        if  self .ctx ._conf .contains ("spark.default.parallelism" ):
163182            return  self .ctx .defaultParallelism 
164183        else :
165184            return  self .getNumPartitions ()
166185
167-         return  self ._jdstream .partitions ().size ()
168- 
169-     def  _defaultReducePartitions (self ):
186+     def  getNumPartitions (self ):
170187        """ 
171- 
188+         Return the number of partitions in RDD 
172189        """ 
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 
187192
188193
189194class  PipelinedDStream (DStream ):
0 commit comments