Skip to content
Closed
Show file tree
Hide file tree
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
5 changes: 4 additions & 1 deletion python/pyspark/streaming/dstream.py
Original file line number Diff line number Diff line change
Expand Up @@ -610,7 +610,10 @@ def __init__(self, prev, func):
self.is_checkpointed = False
self._jdstream_val = None

if (isinstance(prev, TransformedDStream) and
# Using type() to avoid folding the functions and compacting the DStreams which is not
# not strictly a object of TransformedDStream.
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: two nots

# Changed here is to avoid bug in KafkaTransformedDStream when calling offsetRanges().
if (type(prev) is TransformedDStream and
not prev.is_cached and not prev.is_checkpointed):
prev_func = prev.func
self.func = lambda t, rdd: func(t, prev_func(t, rdd))
Expand Down
4 changes: 3 additions & 1 deletion python/pyspark/streaming/tests.py
Original file line number Diff line number Diff line change
Expand Up @@ -850,7 +850,9 @@ def transformWithOffsetRanges(rdd):
offsetRanges.append(o)
return rdd

stream.transform(transformWithOffsetRanges).foreachRDD(lambda rdd: rdd.count())
# Test whether it is ok mixing KafkaTransformedDStream and TransformedDStream together,
# only the TransformedDstreams can be folded together.
stream.transform(transformWithOffsetRanges).map(lambda kv: kv[1]).count().pprint()
self.ssc.start()
self.wait_for(offsetRanges, 1)

Expand Down