aboutsummaryrefslogtreecommitdiff
diff options
context:
space:
mode:
-rw-r--r--python/pyspark/streaming/dstream.py45
1 files changed, 23 insertions, 22 deletions
diff --git a/python/pyspark/streaming/dstream.py b/python/pyspark/streaming/dstream.py
index f61137cb88..b994a53bf2 100644
--- a/python/pyspark/streaming/dstream.py
+++ b/python/pyspark/streaming/dstream.py
@@ -542,31 +542,32 @@ class DStream(object):
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, initialRDD=None):
"""