Skip to content
Closed
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
45 changes: 23 additions & 22 deletions python/pyspark/streaming/dstream.py
Original file line number Diff line number Diff line change
Expand Up @@ -542,31 +542,32 @@ def reduceByKeyAndWindow(self, func, invFunc, windowDuration, slideDuration=None

reduced = self.reduceByKey(func, numPartitions)

def reduceFunc(t, a, b):
b = b.reduceByKey(func, numPartitions)
r = a.union(b).reduceByKey(func, numPartitions) if a else b
if filterFunc:
r = r.filter(filterFunc)
return r

def invReduceFunc(t, a, b):
b = b.reduceByKey(func, numPartitions)
joined = a.leftOuterJoin(b, numPartitions)
return joined.mapValues(lambda kv: invFunc(kv[0], kv[1])
if kv[1] is not None else kv[0])

jreduceFunc = TransformFunction(self._sc, reduceFunc, reduced._jrdd_deserializer)
if invFunc:
def reduceFunc(t, a, b):
b = b.reduceByKey(func, numPartitions)
r = a.union(b).reduceByKey(func, numPartitions) if a else b
if filterFunc:
r = r.filter(filterFunc)
return r

def invReduceFunc(t, a, b):
b = b.reduceByKey(func, numPartitions)
joined = a.leftOuterJoin(b, numPartitions)
return joined.mapValues(lambda kv: invFunc(kv[0], kv[1])
if kv[1] is not None else kv[0])

jreduceFunc = TransformFunction(self._sc, reduceFunc, reduced._jrdd_deserializer)
jinvReduceFunc = TransformFunction(self._sc, invReduceFunc, reduced._jrdd_deserializer)
if slideDuration is None:
slideDuration = self._slideDuration
dstream = self._sc._jvm.PythonReducedWindowedDStream(
reduced._jdstream.dstream(),
jreduceFunc, jinvReduceFunc,
self._ssc._jduration(windowDuration),
self._ssc._jduration(slideDuration))
return DStream(dstream.asJavaDStream(), self._ssc, self._sc.serializer)
else:
jinvReduceFunc = None
if slideDuration is None:
slideDuration = self._slideDuration
dstream = self._sc._jvm.PythonReducedWindowedDStream(reduced._jdstream.dstream(),
jreduceFunc, jinvReduceFunc,
self._ssc._jduration(windowDuration),
self._ssc._jduration(slideDuration))
return DStream(dstream.asJavaDStream(), self._ssc, self._sc.serializer)
return reduced.window(windowDuration, slideDuration).reduceByKey(func, numPartitions)

def updateStateByKey(self, updateFunc, numPartitions=None):
"""
Expand Down