1515# limitations under the License.
1616#
1717
18- from collections import defaultdict
1918from itertools import chain , ifilter , imap
2019import operator
2120
2221from pyspark import RDD
23- from pyspark .serializers import NoOpSerializer ,\
24- BatchedSerializer , CloudPickleSerializer , pack_long ,\
25- CompressedSerializer
2622from pyspark .storagelevel import StorageLevel
27- from pyspark .resultiterable import ResultIterable
28- from pyspark .streaming .util import rddToFileName , RDDFunction
29- from pyspark .rdd import portable_hash , _parse_memory
30- from pyspark .traceback_utils import SCCallSiteSync
23+ from pyspark .streaming .util import rddToFileName , RDDFunction , RDDFunction2
24+ from pyspark .rdd import portable_hash
25+ from pyspark .streaming .duration import Seconds
3126
32- from py4j .java_collections import ListConverter , MapConverter
3327
3428__all__ = ["DStream" ]
3529
@@ -42,7 +36,6 @@ def __init__(self, jdstream, ssc, jrdd_deserializer):
4236 self ._jrdd_deserializer = jrdd_deserializer
4337 self .is_cached = False
4438 self .is_checkpointed = False
45- self ._partitionFunc = None
4639
4740 def context (self ):
4841 """
@@ -159,7 +152,7 @@ def foreachRDD(self, func):
159152 This is an output operator, so this DStream will be registered as an output
160153 stream and there materialized.
161154 """
162- jfunc = RDDFunction (self .ctx , lambda a , b , t : func ( a , t ) , self ._jrdd_deserializer )
155+ jfunc = RDDFunction (self .ctx , func , self ._jrdd_deserializer )
163156 self .ctx ._jvm .PythonForeachDStream (self ._jdstream .dstream (), jfunc )
164157
165158 def pyprint (self ):
@@ -306,19 +299,19 @@ def get_output(rdd, time):
306299 return result
307300
308301 def transform (self , func ):
309- return TransformedRDD (self , lambda a , b , t : func (a ), cache = True )
310-
311- def transformWith (self , func , other ):
312- return TransformedRDD (self , lambda a , b , t : func (a , b ), other )
302+ return TransformedRDD (self , lambda a , t : func (a ), True )
313303
314304 def transformWithTime (self , func ):
315- return TransformedRDD (self , lambda a , b , t : func (a , t ))
305+ return TransformedRDD (self , func , False )
306+
307+ def transformWith (self , func , other , keepSerializer = False ):
308+ return Transformed2RDD (self , lambda a , b , t : func (a , b ), other , keepSerializer )
316309
317310 def repartitions (self , numPartitions ):
318311 return self .transform (lambda rdd : rdd .repartition (numPartitions ))
319312
320313 def union (self , other ):
321- return self .transformWith (lambda a , b : a .union (b ), other )
314+ return self .transformWith (lambda a , b : a .union (b ), other , True )
322315
323316 def cogroup (self , other ):
324317 return self .transformWith (lambda a , b : a .cogroup (b ), other )
@@ -329,32 +322,79 @@ def leftOuterJoin(self, other):
329322 def rightOuterJoin (self , other ):
330323 return self .transformWith (lambda a , b : a .rightOuterJoin (b ), other )
331324
332- def slice (self , fromTime , toTime ):
333- jrdds = self ._jdstream .slice (fromTime ._jtime , toTime ._jtime )
334- # FIXME: serializer
335- return [RDD (jrdd , self .ctx , self .ctx .serializer ) for jrdd in jrdds ]
325+ def _jtime (self , milliseconds ):
326+ return self .ctx ._jvm .Time (milliseconds )
327+
328+ def slice (self , begin , end ):
329+ jrdds = self ._jdstream .slice (self ._jtime (begin ), self ._jtime (end ))
330+ return [RDD (jrdd , self .ctx , self ._jrdd_deserializer ) for jrdd in jrdds ]
331+
332+ def window (self , windowDuration , slideDuration = None ):
333+ d = Seconds (windowDuration )
334+ if slideDuration is None :
335+ return DStream (self ._jdstream .window (d ), self ._ssc , self ._jrdd_deserializer )
336+ s = Seconds (slideDuration )
337+ return DStream (self ._jdstream .window (d , s ), self ._ssc , self ._jrdd_deserializer )
338+
339+ def reduceByWindow (self , reduceFunc , inReduceFunc , windowDuration , slideDuration ):
340+ pass
341+
342+ def countByWindow (self , window , slide ):
343+ pass
344+
345+ def countByValueAndWindow (self , window , slide , numPartitions = None ):
346+ pass
347+
348+ def groupByKeyAndWindow (self , window , slide , numPartitions = None ):
349+ pass
350+
351+ def reduceByKeyAndWindow (self , reduceFunc , inReduceFunc , window , slide , numPartitions = None ):
352+ pass
336353
337354 def updateStateByKey (self , updateFunc ):
338355 # FIXME: convert updateFunc to java JFunction2
339356 jFunc = updateFunc
340357 return self ._jdstream .updateStateByKey (jFunc )
341358
342359
343- # Window Operations
344- # TODO: implement window
345- # TODO: implement groupByKeyAndWindow
346- # TODO: implement reduceByKeyAndWindow
347- # TODO: implement countByValueAndWindow
348- # TODO: implement countByWindow
349- # TODO: implement reduceByWindow
360+ class TransformedRDD (DStream ):
361+ def __init__ (self , prev , func , reuse = False ):
362+ ssc = prev ._ssc
363+ self ._ssc = ssc
364+ self .ctx = ssc ._sc
365+ self ._jrdd_deserializer = self .ctx .serializer
366+ self .is_cached = False
367+ self .is_checkpointed = False
368+
369+ if isinstance (prev , TransformedRDD ) and not prev .is_cached and not prev .is_checkpointed :
370+ prev_func = prev .func
371+ old_func = func
372+ func = lambda rdd , t : old_func (prev_func (rdd , t ), t )
373+ reuse = reuse and prev .reuse
374+ prev = prev .prev
350375
376+ self .prev = prev
377+ self .func = func
378+ self .reuse = reuse
379+ self ._jdstream_val = None
351380
352- class TransformedRDD (DStream ):
353- # TODO: better name for cache
354- def __init__ (self , prev , func , other = None , cache = False ):
355- # TODO: combine transformed RDD
381+ @property
382+ def _jdstream (self ):
383+ if self ._jdstream_val is not None :
384+ return self ._jdstream_val
385+
386+ jfunc = RDDFunction (self .ctx , self .func , self .prev ._jrdd_deserializer )
387+ jdstream = self .ctx ._jvm .PythonTransformedDStream (self .prev ._jdstream .dstream (),
388+ jfunc , self .reuse ).asJavaDStream ()
389+ self ._jdstream_val = jdstream
390+ return jdstream
391+
392+
393+ class Transformed2RDD (DStream ):
394+ def __init__ (self , prev , func , other , keepSerializer = False ):
356395 ssc = prev ._ssc
357- t = RDDFunction (ssc ._sc , func , prev ._jrdd_deserializer )
358- jdstream = ssc ._jvm .PythonTransformedDStream (prev ._jdstream .dstream (),
359- other and other ._jdstream , t , cache )
360- DStream .__init__ (self , jdstream .asJavaDStream (), ssc , ssc ._sc .serializer )
396+ jfunc = RDDFunction2 (ssc ._sc , func , prev ._jrdd_deserializer )
397+ jdstream = ssc ._jvm .PythonTransformed2DStream (prev ._jdstream .dstream (),
398+ other ._jdstream .dstream (), jfunc )
399+ jrdd_serializer = prev ._jrdd_deserializer if keepSerializer else ssc ._sc .serializer
400+ DStream .__init__ (self , jdstream .asJavaDStream (), ssc , jrdd_serializer )
0 commit comments