Skip to content

Commit 79d70ab

Browse files
committed
Added StreamingListenerAdapter for Python.
1 parent 233104d commit 79d70ab

File tree

3 files changed

+114
-20
lines changed

3 files changed

+114
-20
lines changed

python/pyspark/streaming/context.py

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -28,7 +28,7 @@
2828
from pyspark.storagelevel import StorageLevel
2929
from pyspark.streaming.dstream import DStream
3030
from pyspark.streaming.util import TransformFunction, TransformFunctionSerializer
31-
from pyspark.streaming.listener import StreamingListener
31+
from pyspark.streaming.listener import StreamingListenerAdapter
3232

3333
__all__ = ["StreamingContext"]
3434

@@ -370,4 +370,5 @@ def addStreamingListener(self, streamingListener):
370370
Add a [[org.apache.spark.streaming.scheduler.StreamingListener]] object for
371371
receiving system events related to streaming.
372372
"""
373-
self._jssc.addStreamingListener(streamingListener)
373+
streamingListenerAdapter = StreamingListenerAdapter(streamingListener)
374+
self._jssc.addStreamingListener(streamingListenerAdapter)

python/pyspark/streaming/listener.py

Lines changed: 108 additions & 15 deletions
Original file line numberDiff line numberDiff line change
@@ -18,6 +18,68 @@
1818
__all__ = ["StreamingListener"]
1919

2020

21+
class StreamingListenerEvent(object):
22+
23+
def __init__(self):
24+
pass
25+
26+
27+
class StreamingListenerBatchSubmitted(StreamingListenerEvent):
28+
29+
def __init__(self, batchInfo):
30+
super(StreamingListenerEvent, self).__init__()
31+
self.batchInfo = batchInfo
32+
33+
34+
class StreamingListenerBatchCompleted(StreamingListenerEvent):
35+
36+
def __init__(self, batchInfo):
37+
super(StreamingListenerEvent, self).__init__()
38+
self.batchInfo = batchInfo
39+
40+
41+
class StreamingListenerBatchStarted(StreamingListenerEvent):
42+
43+
def __init__(self, batchInfo):
44+
super(StreamingListenerEvent, self).__init__()
45+
self.batchInfo = batchInfo
46+
47+
48+
class StreamingListenerOutputOperationStarted(StreamingListenerEvent):
49+
50+
def __init__(self, outputOperationInfo):
51+
super(StreamingListenerEvent, self).__init__()
52+
self.outputOperationInfo = outputOperationInfo
53+
54+
55+
class StreamingListenerOutputOperationCompleted(StreamingListenerEvent):
56+
57+
def __init__(self, outputOperationInfo):
58+
super(StreamingListenerEvent, self).__init__()
59+
self.outputOperationInfo = outputOperationInfo
60+
61+
62+
class StreamingListenerReceieverStarted(StreamingListenerEvent):
63+
64+
def __init__(self, receiverInfo):
65+
super(StreamingListenerEvent, self).__init__()
66+
self.receiverInfo = receiverInfo
67+
68+
69+
class StreamingListenerReceiverError(StreamingListenerEvent):
70+
71+
def __init__(self, receiverInfo):
72+
super(StreamingListenerEvent, self).__init__()
73+
self.receiverInfo = receiverInfo
74+
75+
76+
class StreamingListenerReceiverStopped(StreamingListenerEvent):
77+
78+
def __init__(self, receiverInfo):
79+
super(StreamingListenerEvent, self).__init__()
80+
self.receiverInfo = receiverInfo
81+
82+
2183
class StreamingListener(object):
2284

2385
def __init__(self):
@@ -71,24 +133,55 @@ def onOutputOperationCompleted(self, outputOperationCompleted):
71133
"""
72134
pass
73135

74-
def getEventInfo(self, event):
75-
"""
76-
:param event: StreamingListenerEvent
77-
:return Returns a BatchInfo, OutputOperationInfo, or ReceiverInfo based on
78-
event passed.
79-
"""
80-
event_name = event.getClass().getSimpleName()
81-
if 'Batch' in event_name:
82-
return BatchInfo(event.batchInfo())
136+
class Java:
137+
implements = ["org.apache.spark.streaming.scheduler.StreamingListener"]
83138

84-
elif 'Output' in event_name:
85-
return OutputOperationInfo(event.outputOperationInfo())
86139

87-
elif 'Receiver' in event_name:
88-
return ReceiverInfo(event.receiverInfo())
140+
class StreamingListenerAdapter(StreamingListener):
89141

90-
class Java:
91-
implements = ["org.apache.spark.streaming.scheduler.StreamingListener"]
142+
def __init__(self, streamingListener):
143+
super(StreamingListener, self).__init__()
144+
self.userStreamingListener = streamingListener
145+
146+
def onReceiverStarted(self, receiverStarted):
147+
receiver_info = ReceiverInfo(receiverStarted.receiverInfo())
148+
receiver_started = StreamingListenerReceieverStarted(receiver_info)
149+
self.userStreamingListener.onReceiverStarted(receiver_started)
150+
151+
def onReceiverError(self, receiverError):
152+
receiver_info = ReceiverInfo(receiverError.receiverInfo())
153+
receiver_error = StreamingListenerReceiverError(receiver_info)
154+
self.userStreamingListener.onReceiverError(receiver_error)
155+
156+
def onReceiverStopped(self, receiverStopped):
157+
receiver_info = ReceiverInfo(receiverStopped.receiverInfo())
158+
receiver_stopped = StreamingListenerReceiverStopped(receiver_info)
159+
self.userStreamingListener.onReceiverStopped(receiver_stopped)
160+
161+
def onBatchSubmitted(self, batchSubmitted):
162+
batch_info = BatchInfo(batchSubmitted.batchInfo())
163+
batch_submitted = StreamingListenerBatchSubmitted(batch_info)
164+
self.userStreamingListener.onBatchSubmitted(batch_submitted)
165+
166+
def onBatchStarted(self, batchStarted):
167+
batch_info = BatchInfo(batchStarted.batchInfo())
168+
batch_started = StreamingListenerBatchStarted(batch_info)
169+
self.userStreamingListener .onBatchStarted(batch_started)
170+
171+
def onBatchCompleted(self, batchCompleted):
172+
batch_info = BatchInfo(batchCompleted.batchInfo())
173+
batch_completed = StreamingListenerBatchCompleted(batch_info)
174+
self.userStreamingListener.onBatchCompleted(batch_completed)
175+
176+
def onOutputOperationStarted(self, outputOperationStarted):
177+
output_op_info = OutputOperationInfo(outputOperationStarted.outputOperationInfo())
178+
output_operation_started = StreamingListenerOutputOperationStarted(output_op_info)
179+
self.userStreamingListener.onOutputOperationStarted(output_operation_started)
180+
181+
def onOutputOperationCompleted(self, outputOperationCompleted):
182+
output_op_info = OutputOperationInfo(outputOperationCompleted.outputOperationInfo())
183+
output_operation_completed = StreamingListenerOutputOperationCompleted(output_op_info)
184+
self.userStreamingListener.onOutputOperationCompleted(output_operation_completed)
92185

93186

94187
class BatchInfo(object):

python/pyspark/streaming/tests.py

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -412,13 +412,13 @@ def __init__(self):
412412
self.batchInfosSubmitted = []
413413

414414
def onBatchSubmitted(self, batchSubmitted):
415-
self.batchInfosSubmitted.append(self.getEventInfo(batchSubmitted))
415+
self.batchInfosSubmitted.append(batchSubmitted.batchInfo)
416416

417417
def onBatchStarted(self, batchStarted):
418-
self.batchInfosStarted.append(self.getEventInfo(batchStarted))
418+
self.batchInfosStarted.append(batchStarted.batchInfo)
419419

420420
def onBatchCompleted(self, batchCompleted):
421-
self.batchInfosCompleted.append(self.getEventInfo(batchCompleted))
421+
self.batchInfosCompleted.append(batchCompleted.batchInfo)
422422

423423
def test_batch_info_reports(self):
424424
batch_collector = self.BatchInfoCollector()

0 commit comments

Comments
 (0)